From 9032e23f87d2f8ed88ad62c79017c2c951d74464 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 May 2023 15:43:23 +0200 Subject: [PATCH 001/473] Clean up storage_conf.xml, use dynamic disks for tests --- tests/config/config.d/storage_conf.xml | 133 ------------------ ...ilesystem_cache_bypass_cache_threshold.sql | 19 ++- .../02240_filesystem_query_cache.sql | 18 ++- .../02344_describe_cache.reference | 1 - .../0_stateless/02344_describe_cache.sql | 3 - ..._cache_on_write_with_small_segment_size.sh | 17 ++- 6 files changed, 51 insertions(+), 140 deletions(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index dee03307177..98798d5746e 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -10,46 +10,6 @@ clickhouse 20000 - - s3 - s3_disk_2/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_3/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_4/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_5/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_6/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - cache @@ -60,54 +20,6 @@ 0 100 - - cache - s3_disk_2 - s3_cache_2/ - 128Mi - 0 - 100Mi - 100 - - - cache - s3_disk_3 - s3_disk_3_cache/ - 128Mi - 22548578304 - 1 - 1 - 0 - 100 - - - cache - s3_disk_4 - s3_cache_4/ - 128Mi - 1 - 1 - 0 - 100 - - - cache - s3_disk_5 - s3_cache_5/ - 128Mi - 0 - 100 - - - cache - s3_disk_6 - s3_cache_6/ - 128Mi - 0 - 1 - 100 - 100 - cache s3_disk_6 @@ -116,16 +28,6 @@ 1 100 - - cache - s3_disk_6 - s3_cache_small_segment_size/ - 128Mi - 10Ki - 0 - 1 - 100 - local_blob_storage @@ -193,34 +95,6 @@ - - -
- s3_cache_2 -
-
-
- - -
- s3_cache_3 -
-
-
- - -
- s3_cache_4 -
-
-
- - -
- s3_cache_6 -
-
-
@@ -256,13 +130,6 @@
- - -
- s3_cache_small_segment_size -
-
-
diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql index f6671b82291..ae2cd1b8cd1 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql @@ -6,7 +6,24 @@ SYSTEM DROP FILESYSTEM CACHE; SET enable_filesystem_cache_on_write_operations=0; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + enable_bypass_cache_with_threashold = 1, + bypass_cache_threashold = 100, + cache_on_write_operations = 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 94eb4bc5ccd..f2664de5c63 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -8,7 +8,23 @@ SET skip_download_if_exceeds_query_cache=1; SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; + +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations= 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7561b32bae1..4302b05e136 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1 @@ 134217728 1048576 104857600 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 -134217728 1048576 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 diff --git a/tests/queries/0_stateless/02344_describe_cache.sql b/tests/queries/0_stateless/02344_describe_cache.sql index a687ad01394..9c5c5c10952 100644 --- a/tests/queries/0_stateless/02344_describe_cache.sql +++ b/tests/queries/0_stateless/02344_describe_cache.sql @@ -1,7 +1,4 @@ -- Tags: no-fasttest, no-parallel SYSTEM DROP FILESYSTEM CACHE 's3_cache'; -SYSTEM DROP FILESYSTEM CACHE 's3_cache_2'; - DESCRIBE FILESYSTEM CACHE 's3_cache'; -DESCRIBE FILESYSTEM CACHE 's3_cache_2'; diff --git a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh index ed66c36b823..2c526d10cc9 100755 --- a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh +++ b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh @@ -13,7 +13,22 @@ function random { ${CLICKHOUSE_CLIENT} --multiline --multiquery -q " drop table if exists ttt; -create table ttt (id Int32, value String) engine=MergeTree() order by tuple() settings storage_policy='s3_cache_small_segment_size', min_bytes_for_wide_part=0; + +CREATE TABLE ttt (id Int32, value String) +Engine=MergeTree() +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, + disk = disk( + type = cache, + max_size = '128Mi', + max_file_segment_size = '10Ki', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations = 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + insert into ttt select number, toString(number) from numbers(100000) settings throw_on_error_from_cache_on_write_operations = 1; " From 431b2e94a936f5021b6d834994614d8d03ddafcb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 5 May 2023 19:10:12 +0200 Subject: [PATCH 002/473] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 98798d5746e..923240d5a91 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -22,7 +22,7 @@ cache - s3_disk_6 + s3_disk s3_cache_small/ 1000 1 From 8c0b634a644ac85832658cc6ca863909ef455795 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 7 May 2023 14:38:27 +0200 Subject: [PATCH 003/473] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 923240d5a91..aad93a017c2 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -72,7 +72,7 @@ cache - s3_cache_5 + s3_cache s3_cache_multi/ 22548578304 0 From 383fc06761f81bee735ec22692a2d506ca78c01e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 9 May 2023 16:10:53 +0200 Subject: [PATCH 004/473] Fix --- .../MaterializedPostgreSQLConsumer.cpp | 37 +++--- .../MaterializedPostgreSQLSettings.h | 3 + .../PostgreSQLReplicationHandler.cpp | 21 +-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 7 +- tests/integration/helpers/postgres_utility.py | 124 +++++++++++------- .../test.py | 75 ++++++++++- 6 files changed, 183 insertions(+), 84 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index d048c94ac75..ea7009fc082 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -562,34 +562,27 @@ void MaterializedPostgreSQLConsumer::syncTables() Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); - try + if (result_rows.rows()) { - if (result_rows.rows()) - { - auto storage = storage_data.storage; + auto storage = storage_data.storage; - auto insert_context = Context::createCopy(context); - insert_context->setInternalQuery(true); + auto insert_context = Context::createCopy(context); + insert_context->setInternalQuery(true); - auto insert = std::make_shared(); - insert->table_id = storage->getStorageID(); - insert->columns = storage_data.buffer.columns_ast; + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + insert->columns = storage_data.buffer.columns_ast; - InterpreterInsertQuery interpreter(insert, insert_context, true); - auto io = interpreter.execute(); - auto input = std::make_shared( - result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); + InterpreterInsertQuery interpreter(insert, insert_context, true); + auto io = interpreter.execute(); + auto input = std::make_shared( + result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); - io.pipeline.complete(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); + io.pipeline.complete(Pipe(std::move(input))); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index e8d42ef3668..d3d2faba497 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -21,6 +21,9 @@ namespace DB M(Bool, materialized_postgresql_tables_list_with_schema, false, \ "Consider by default that if there is a dot in tables list 'name.name', " \ "then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \ + M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \ + M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \ + M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 998db4ea79e..f57a6a26a62 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -22,8 +22,6 @@ namespace DB { -static const auto RESCHEDULE_MS = 1000; -static const auto BACKOFF_TRESHOLD_MS = 10000; static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min namespace ErrorCodes @@ -80,7 +78,10 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , schema_list(replication_settings.materialized_postgresql_schema_list) , schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) - , milliseconds_to_wait(RESCHEDULE_MS) + , reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms) + , reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms) + , reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor) + , milliseconds_to_wait(reschedule_backoff_min_ms) { if (!schema_list.empty() && !tables_list.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and tables list at the same time"); @@ -166,7 +167,7 @@ void PostgreSQLReplicationHandler::checkConnectionAndStart() throw; LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what()); - startup_task->scheduleAfter(RESCHEDULE_MS); + startup_task->scheduleAfter(milliseconds_to_wait); } catch (...) { @@ -435,18 +436,18 @@ void PostgreSQLReplicationHandler::consumerFunc() if (schedule_now) { - milliseconds_to_wait = RESCHEDULE_MS; + milliseconds_to_wait = reschedule_backoff_min_ms; consumer_task->schedule(); LOG_DEBUG(log, "Scheduling replication thread: now"); } else { - consumer_task->scheduleAfter(milliseconds_to_wait); - if (milliseconds_to_wait < BACKOFF_TRESHOLD_MS) - milliseconds_to_wait *= 2; + if (milliseconds_to_wait < reschedule_backoff_max_ms) + milliseconds_to_wait = std::min(milliseconds_to_wait * reschedule_backoff_factor, reschedule_backoff_max_ms); LOG_DEBUG(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); + consumer_task->scheduleAfter(milliseconds_to_wait); } } @@ -892,7 +893,7 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost catch (...) { consumer_task->activate(); - consumer_task->scheduleAfter(RESCHEDULE_MS); + consumer_task->scheduleAfter(milliseconds_to_wait); auto error_message = getCurrentExceptionMessage(false); throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, @@ -922,7 +923,7 @@ void PostgreSQLReplicationHandler::removeTableFromReplication(const String & pos catch (...) { consumer_task->activate(); - consumer_task->scheduleAfter(RESCHEDULE_MS); + consumer_task->scheduleAfter(milliseconds_to_wait); auto error_message = getCurrentExceptionMessage(false); throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 10a196cf31b..4c16ff95692 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -140,13 +140,16 @@ private: BackgroundSchedulePool::TaskHolder consumer_task; BackgroundSchedulePool::TaskHolder cleanup_task; + const UInt64 reschedule_backoff_min_ms; + const UInt64 reschedule_backoff_max_ms; + const UInt64 reschedule_backoff_factor; + UInt64 milliseconds_to_wait; + std::atomic stop_synchronization = false; /// MaterializedPostgreSQL tables. Used for managing all operations with its internal nested tables. MaterializedStorages materialized_storages; - UInt64 milliseconds_to_wait; - bool replication_handler_initialized = false; }; diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index dfae37af434..1a00faf0f9d 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -76,16 +76,24 @@ def drop_postgres_schema(cursor, schema_name): def create_postgres_table( - cursor, table_name, replica_identity_full=False, template=postgres_table_template + cursor, + table_name, + database_name="", + replica_identity_full=False, + template=postgres_table_template, ): - drop_postgres_table(cursor, table_name) - cursor.execute(template.format(table_name)) + if database_name == "": + name = table_name + else: + name = f"{database_name}.{table_name}" + drop_postgres_table(cursor, name) + cursor.execute(template.format(name)) if replica_identity_full: - cursor.execute(f"ALTER TABLE {table_name} REPLICA IDENTITY FULL;") + cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") -def drop_postgres_table(cursor, table_name): - cursor.execute(f"""DROP TABLE IF EXISTS "{table_name}" """) +def drop_postgres_table(cursor, name): + cursor.execute(f"""DROP TABLE IF EXISTS "{name}" """) def create_postgres_table_with_schema(cursor, schema_name, table_name): @@ -103,13 +111,16 @@ class PostgresManager: self.created_materialized_postgres_db_list = set() self.created_ch_postgres_db_list = set() - def init(self, instance, ip, port): + def init(self, instance, ip, port, default_database="postgres_database"): self.instance = instance self.ip = ip self.port = port - self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.default_database = default_database self.prepare() + def get_default_database(self): + return self.default_database + def restart(self): try: self.clear() @@ -119,10 +130,17 @@ class PostgresManager: raise ex def prepare(self): - conn = get_postgres_conn(ip=self.ip, port=self.port) - cursor = conn.cursor() - self.create_postgres_db(cursor, "postgres_database") - self.create_clickhouse_postgres_db(ip=self.ip, port=self.port) + self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.cursor = self.conn.cursor() + if self.default_database != "": + self.create_postgres_db(self.default_database) + self.conn = get_postgres_conn( + ip=self.ip, + port=self.port, + database=True, + database_name=self.default_database, + ) + self.cursor = self.conn.cursor() def clear(self): if self.conn.closed == 0: @@ -132,63 +150,76 @@ class PostgresManager: for db in self.created_ch_postgres_db_list.copy(): self.drop_clickhouse_postgres_db(db) if len(self.created_postgres_db_list) > 0: - conn = get_postgres_conn(ip=self.ip, port=self.port) - cursor = conn.cursor() + self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.cursor = self.conn.cursor() for db in self.created_postgres_db_list.copy(): - self.drop_postgres_db(cursor, db) + self.drop_postgres_db(db) - def get_db_cursor(self): - self.conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) + def get_db_cursor(self, database_name=""): + if database_name == "": + database_name = self.default_database + self.conn = get_postgres_conn( + ip=self.ip, port=self.port, database=True, database_name=database_name + ) return self.conn.cursor() - def create_postgres_db(self, cursor, name="postgres_database"): - self.drop_postgres_db(cursor, name) - self.created_postgres_db_list.add(name) - cursor.execute(f"CREATE DATABASE {name}") + def database_or_default(self, database_name): + if database_name == "" and self.default_database == "": + raise Exception("Database name is empty") + if database_name == "": + database_name = self.default_database + return database_name - def drop_postgres_db(self, cursor, name="postgres_database"): - cursor.execute(f"DROP DATABASE IF EXISTS {name}") - if name in self.created_postgres_db_list: - self.created_postgres_db_list.remove(name) + def create_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.drop_postgres_db(database_name) + self.created_postgres_db_list.add(database_name) + self.cursor.execute(f"CREATE DATABASE {database_name}") + + def drop_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name}") + if database_name in self.created_postgres_db_list: + self.created_postgres_db_list.remove(database_name) def create_clickhouse_postgres_db( self, - ip, - port, - name="postgres_database", - database_name="postgres_database", + database_name="", schema_name="", ): - self.drop_clickhouse_postgres_db(name) - self.created_ch_postgres_db_list.add(name) + database_name = self.database_or_default(database_name) + self.drop_clickhouse_postgres_db(database_name) + self.created_ch_postgres_db_list.add(database_name) if len(schema_name) == 0: self.instance.query( f""" - CREATE DATABASE {name} - ENGINE = PostgreSQL('{ip}:{port}', '{database_name}', 'postgres', 'mysecretpassword')""" + CREATE DATABASE {database_name} + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" - CREATE DATABASE {name} - ENGINE = PostgreSQL('{ip}:{port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" + CREATE DATABASE {database_name} + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) - def drop_clickhouse_postgres_db(self, name="postgres_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {name}") - if name in self.created_ch_postgres_db_list: - self.created_ch_postgres_db_list.remove(name) + def drop_clickhouse_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.instance.query(f"DROP DATABASE IF EXISTS {database_name}") + if database_name in self.created_ch_postgres_db_list: + self.created_ch_postgres_db_list.remove(database_name) def create_materialized_db( self, ip, port, materialized_database="test_database", - postgres_database="postgres_database", + postgres_database="", settings=[], table_overrides="", ): + postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") @@ -207,17 +238,12 @@ class PostgresManager: self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) - assert materialized_database not in self.instance.query("SHOW DATABASES") - def create_and_fill_postgres_table(self, table_name): - conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) - cursor = conn.cursor() - self.create_and_fill_postgres_table_from_cursor(cursor, table_name) - - def create_and_fill_postgres_table_from_cursor(self, cursor, table_name): - create_postgres_table(cursor, table_name) + def create_and_fill_postgres_table(self, table_name, database_name=""): + create_postgres_table(self.cursor, table_name, database_name) + database_name = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT number, number from numbers(50)" + f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) def create_and_fill_postgres_tables(self, tables_num, numbers=50): diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 90d19e9532c..3b5194e8806 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -46,14 +46,34 @@ instance = cluster.add_instance( stay_alive=True, ) +instance2 = cluster.add_instance( + "instance2", + main_configs=["configs/log_conf.xml", "configs/merge_tree_too_many_parts.xml"], + user_configs=["configs/users.xml"], + with_postgres=True, + stay_alive=True, +) + + pg_manager = PostgresManager() +pg_manager2 = PostgresManager() @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - pg_manager.init(instance, cluster.postgres_ip, cluster.postgres_port) + pg_manager.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="test_database", + ) + pg_manager.create_clickhouse_postgres_db() + pg_manager2.init( + instance2, cluster.postgres_ip, cluster.postgres_port, "test_database2" + ) + pg_manager2.create_clickhouse_postgres_db() yield cluster finally: @@ -649,6 +669,59 @@ def test_materialized_view(started_cluster): pg_manager.drop_materialized_db() +def test_too_many_parts(started_cluster): + table = "test_table" + pg_manager2.create_and_fill_postgres_table(table) + pg_manager2.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = 'test_table', materialized_postgresql_backoff_min_ms = 100, materialized_postgresql_backoff_max_ms = 100" + ], + ) + check_tables_are_synchronized( + instance2, "test_table", postgres_database=pg_manager2.get_default_database() + ) + assert ( + "50" == instance2.query("SELECT count() FROM test_database.test_table").strip() + ) + + instance2.query("SYSTEM STOP MERGES") + num = 50 + for i in range(10): + instance2.query( + f""" + INSERT INTO {pg_manager2.get_default_database()}.test_table SELECT {num}, {num}; + """ + ) + num = num + 1 + for i in range(30): + if num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) or instance2.contains_in_log("DB::Exception: Too many parts"): + break + time.sleep(1) + print(f"wait sync try {i}") + if instance2.contains_in_log("DB::Exception: Too many parts"): + num = num - 1 + break + assert num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) + + assert instance2.contains_in_log("DB::Exception: Too many parts") + print(num) + assert num == int(instance2.query("SELECT count() FROM test_database.test_table")) + + instance2.query("SYSTEM START MERGES") + check_tables_are_synchronized( + instance2, "test_table", postgres_database=pg_manager2.get_default_database() + ) + + # assert "200" == instance.query("SELECT count FROM test_database.test_table").strip() + pg_manager2.drop_materialized_db() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From c9e752fdc5c4cc401df240f7cd5f77586d9b542d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 11 May 2023 18:09:46 +0200 Subject: [PATCH 005/473] Fix --- tests/integration/helpers/postgres_utility.py | 51 ++++---- .../test.py | 113 +++++++----------- .../test.py | 64 +++++----- 3 files changed, 99 insertions(+), 129 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 1a00faf0f9d..3c8a23b15a2 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -87,7 +87,9 @@ def create_postgres_table( else: name = f"{database_name}.{table_name}" drop_postgres_table(cursor, name) - cursor.execute(template.format(name)) + query = template.format(name) + cursor.execute(query) + print(f"Query: {query}") if replica_identity_full: cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") @@ -129,6 +131,9 @@ class PostgresManager: self.prepare() raise ex + def execute(self, query): + self.cursor.execute(query) + def prepare(self): self.conn = get_postgres_conn(ip=self.ip, port=self.port) self.cursor = self.conn.cursor() @@ -141,6 +146,7 @@ class PostgresManager: database_name=self.default_database, ) self.cursor = self.conn.cursor() + self.create_clickhouse_postgres_db() def clear(self): if self.conn.closed == 0: @@ -164,11 +170,11 @@ class PostgresManager: return self.conn.cursor() def database_or_default(self, database_name): - if database_name == "" and self.default_database == "": - raise Exception("Database name is empty") - if database_name == "": - database_name = self.default_database - return database_name + if database_name != "": + return database_name + if self.default_database != "": + return self.default_database + raise Exception("Database name is empty") def create_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) @@ -186,8 +192,11 @@ class PostgresManager: self, database_name="", schema_name="", + postgres_database="", ): database_name = self.database_or_default(database_name) + if postgres_database == "": + postgres_database = database_name self.drop_clickhouse_postgres_db(database_name) self.created_ch_postgres_db_list.add(database_name) @@ -195,13 +204,13 @@ class PostgresManager: self.instance.query( f""" CREATE DATABASE {database_name} - ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword')""" + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" CREATE DATABASE {database_name} - ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) def drop_clickhouse_postgres_db(self, database_name=""): @@ -239,6 +248,16 @@ class PostgresManager: if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) + def create_postgres_schema(self, name): + create_postgres_schema(self.cursor, name) + + def create_postgres_table( + self, table_name, database_name="", template=postgres_table_template + ): + create_postgres_table( + self.cursor, table_name, database_name=database_name, template=template + ) + def create_and_fill_postgres_table(self, table_name, database_name=""): create_postgres_table(self.cursor, table_name, database_name) database_name = self.database_or_default(database_name) @@ -246,22 +265,14 @@ class PostgresManager: f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) - def create_and_fill_postgres_tables(self, tables_num, numbers=50): - conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) - cursor = conn.cursor() - self.create_and_fill_postgres_tables_from_cursor( - cursor, tables_num, numbers=numbers - ) - - def create_and_fill_postgres_tables_from_cursor( - self, cursor, tables_num, numbers=50 - ): + def create_and_fill_postgres_tables(self, tables_num, numbers=50, database_name=""): for i in range(tables_num): table_name = f"postgresql_replica_{i}" - create_postgres_table(cursor, table_name) + create_postgres_table(self.cursor, table_name, database_name) if numbers > 0: + db = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT number, number from numbers({numbers})" + f"INSERT INTO {db}.{table_name} SELECT number, number from numbers({numbers})" ) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index 377b1c89efc..1eb2efc73a5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -46,7 +46,12 @@ pg_manager = PostgresManager() def started_cluster(): try: cluster.start() - pg_manager.init(instance, cluster.postgres_ip, cluster.postgres_port) + pg_manager.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + ) yield cluster finally: @@ -74,16 +79,10 @@ def test_load_and_sync_all_database_tables(started_cluster): def test_replicating_dml(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table(cursor, "postgresql_replica_{}".format(i)) + pg_manager.create_postgres_table(f"postgresql_replica_{i}") instance.query( "INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format( i, i @@ -96,39 +95,29 @@ def test_replicating_dml(started_cluster): for i in range(NUM_TABLES): instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format( - i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT 50 + number, {i} from numbers(1000)" ) check_several_tables_are_synchronized(instance, NUM_TABLES) for i in range(NUM_TABLES): - cursor.execute( - "UPDATE postgresql_replica_{} SET value = {} * {} WHERE key < 50;".format( - i, i, i - ) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = {i} * {i} WHERE key < 50;" ) - cursor.execute( - "UPDATE postgresql_replica_{} SET value = {} * {} * {} WHERE key >= 50;".format( - i, i, i, i - ) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = {i} * {i} * {i} WHERE key >= 50;" ) + check_several_tables_are_synchronized(instance, NUM_TABLES) for i in range(NUM_TABLES): - cursor.execute( - "DELETE FROM postgresql_replica_{} WHERE (value*value + {}) % 2 = 0;".format( - i, i - ) + pg_manager.execute( + f"DELETE FROM postgresql_replica_{i} WHERE (value*value + {i}) % 2 = 0;" ) - cursor.execute( - "UPDATE postgresql_replica_{} SET value = value - (value % 7) WHERE key > 128 AND key < 512;".format( - i - ) - ) - cursor.execute( - "DELETE FROM postgresql_replica_{} WHERE key % 7 = 1;".format(i, i) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = value - (value % 7) WHERE key > 128 AND key < 512;" ) + pg_manager.execute(f"DELETE FROM postgresql_replica_{i} WHERE key % 7 = 1;") + check_several_tables_are_synchronized(instance, NUM_TABLES) @@ -288,13 +277,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): def test_changing_replica_identity_value(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") + pg_manager.create_postgres_table("postgresql_replica") instance.query( "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)" ) @@ -307,7 +290,7 @@ def test_changing_replica_identity_value(started_cluster): "INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)" ) check_tables_are_synchronized(instance, "postgresql_replica") - cursor.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") + pg_manager.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") check_tables_are_synchronized(instance, "postgresql_replica") @@ -331,18 +314,13 @@ def test_clickhouse_restart(started_cluster): def test_replica_identity_index(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, + pg_manager.create_postgres_table( + "postgresql_replica", template=postgres_table_template_3 ) - cursor = conn.cursor() - - create_postgres_table( - cursor, "postgresql_replica", template=postgres_table_template_3 + pg_manager.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);") + pg_manager.execute( + "ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx" ) - cursor.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);") - cursor.execute("ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx") instance.query( "INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(50, 10)" ) @@ -355,35 +333,29 @@ def test_replica_identity_index(started_cluster): ) check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") - cursor.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ") - cursor.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ") - cursor.execute("UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 ") - cursor.execute("UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 ") + pg_manager.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ") + pg_manager.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ") + pg_manager.execute( + "UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 " + ) + pg_manager.execute( + "UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 " + ) check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") - cursor.execute("DELETE FROM postgresql_replica WHERE key2<75;") + pg_manager.execute("DELETE FROM postgresql_replica WHERE key2<75;") check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") def test_table_schema_changes(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table( - cursor, - "postgresql_replica_{}".format(i), - template=postgres_table_template_2, + pg_manager.create_postgres_table( + f"postgresql_replica_{i}", template=postgres_table_template_2 ) instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format( - i, i, i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT number, {i}, {i}, {i} from numbers(25)" ) pg_manager.create_materialized_db( @@ -393,9 +365,7 @@ def test_table_schema_changes(started_cluster): for i in range(NUM_TABLES): instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT 25 + number, {}, {}, {} from numbers(25)".format( - i, i, i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT 25 + number, {i}, {i}, {i} from numbers(25)" ) check_several_tables_are_synchronized(instance, NUM_TABLES) @@ -444,10 +414,7 @@ def test_many_concurrent_queries(started_cluster): port=started_cluster.postgres_port, database=True, ) - cursor = conn.cursor() - pg_manager.create_and_fill_postgres_tables_from_cursor( - cursor, NUM_TABLES, numbers=10000 - ) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000) def attack(thread_id): print("thread {}".format(thread_id)) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3b5194e8806..2b17024f417 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -67,13 +67,11 @@ def started_cluster(): instance, cluster.postgres_ip, cluster.postgres_port, - default_database="test_database", + default_database="postgres_database", ) - pg_manager.create_clickhouse_postgres_db() pg_manager2.init( - instance2, cluster.postgres_ip, cluster.postgres_port, "test_database2" + instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) - pg_manager2.create_clickhouse_postgres_db() yield cluster finally: @@ -88,11 +86,10 @@ def setup_teardown(): def test_add_new_table_to_replication(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute("DROP TABLE IF EXISTS test_table") + pg_manager.execute("DROP TABLE IF EXISTS test_table") NUM_TABLES = 5 - pg_manager.create_and_fill_postgres_tables_from_cursor(cursor, NUM_TABLES, 10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, 10000) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) @@ -105,7 +102,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_5" - pg_manager.create_and_fill_postgres_table_from_cursor(cursor, table_name) + pg_manager.create_and_fill_postgres_table(table_name) result = instance.query("SHOW CREATE DATABASE test_database") assert ( @@ -158,7 +155,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_6" - create_postgres_table(cursor, table_name) + pg_manager.create_postgres_table(table_name) instance.query( "INSERT INTO postgres_database.{} SELECT number, number from numbers(10000)".format( table_name @@ -169,7 +166,7 @@ def test_add_new_table_to_replication(started_cluster): instance.restart_clickhouse() table_name = "postgresql_replica_7" - create_postgres_table(cursor, table_name) + pg_manager.create_postgres_table(table_name) instance.query( "INSERT INTO postgres_database.{} SELECT number, number from numbers(10000)".format( table_name @@ -271,8 +268,7 @@ def test_remove_table_from_replication(started_cluster): == ")\\nSETTINGS materialized_postgresql_tables_list = \\'postgresql_replica_0,postgresql_replica_2,postgresql_replica_3,postgresql_replica_4\\'\n" ) - cursor = pg_manager.get_db_cursor() - cursor.execute(f"drop table if exists postgresql_replica_0;") + pg_manager.execute(f"drop table if exists postgresql_replica_0;") # Removing from replication table which does not exist in PostgreSQL must be ok. instance.query("DETACH TABLE test_database.postgresql_replica_0 PERMANENTLY") @@ -282,10 +278,11 @@ def test_remove_table_from_replication(started_cluster): def test_predefined_connection_configuration(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute(f"DROP TABLE IF EXISTS test_table") - cursor.execute(f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)") - cursor.execute(f"INSERT INTO test_table SELECT 1, 2") + pg_manager.execute(f"DROP TABLE IF EXISTS test_table") + pg_manager.execute( + f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)" + ) + pg_manager.execute(f"INSERT INTO test_table SELECT 1, 2") instance.query( "CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'" ) @@ -332,10 +329,9 @@ def test_database_with_single_non_default_schema(started_cluster): create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, + database_name=clickhouse_postgres_db, schema_name=schema_name, + postgres_database="postgres_database", ) for i in range(NUM_TABLES): @@ -367,7 +363,7 @@ def test_database_with_single_non_default_schema(started_cluster): check_all_tables_are_synchronized() altered_table = random.randint(0, NUM_TABLES - 1) - cursor.execute( + pg_manager.execute( "ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format( altered_table ) @@ -434,10 +430,9 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, + database_name=clickhouse_postgres_db, schema_name=schema_name, + postgres_database="postgres_database", ) for i in range(NUM_TABLES): @@ -472,7 +467,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): check_all_tables_are_synchronized() altered_table = random.randint(0, NUM_TABLES - 1) - cursor.execute( + pg_manager.execute( "ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format( altered_table ) @@ -550,10 +545,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, - schema_name=schema_name, + database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" @@ -586,7 +578,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): altered_schema = random.randint(0, schemas_num - 1) altered_table = random.randint(0, NUM_TABLES - 1) clickhouse_postgres_db = f"clickhouse_postgres_db{altered_schema}" - cursor.execute( + pg_manager.execute( f"ALTER TABLE schema{altered_schema}.postgresql_replica_{altered_table} ADD COLUMN value2 integer" ) @@ -619,10 +611,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): def test_table_override(started_cluster): - cursor = pg_manager.get_db_cursor() table_name = "table_override" materialized_database = "test_database" - create_postgres_table(cursor, table_name, template=postgres_table_template_5) + pg_manager.create_postgres_table(table_name, template=postgres_table_template_5) instance.query( f"create table {table_name}(key Int32, value UUID) engine = PostgreSQL (postgres1, table={table_name})" ) @@ -649,10 +640,11 @@ def test_table_override(started_cluster): def test_materialized_view(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute(f"DROP TABLE IF EXISTS test_table") - cursor.execute(f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)") - cursor.execute(f"INSERT INTO test_table SELECT 1, 2") + pg_manager.execute(f"DROP TABLE IF EXISTS test_table") + pg_manager.execute( + f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)" + ) + pg_manager.execute(f"INSERT INTO test_table SELECT 1, 2") instance.query("DROP DATABASE IF EXISTS test_database") instance.query( "CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'" @@ -663,7 +655,7 @@ def test_materialized_view(started_cluster): "CREATE MATERIALIZED VIEW mv ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT * FROM test_database.test_table" ) assert "1\t2" == instance.query("SELECT * FROM mv").strip() - cursor.execute(f"INSERT INTO test_table SELECT 3, 4") + pg_manager.execute(f"INSERT INTO test_table SELECT 3, 4") check_tables_are_synchronized(instance, "test_table") assert "1\t2\n3\t4" == instance.query("SELECT * FROM mv ORDER BY 1, 2").strip() pg_manager.drop_materialized_db() From e2112576f04d3aeda1bc5384b5c49da4dc2a8e0c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 16 May 2023 12:21:32 +0200 Subject: [PATCH 006/473] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2b17024f417..acec01e732b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -545,7 +545,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", + database_name=clickhouse_postgres_db, + schema_name=schema_name, + postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" From 849cddd8b281be1ecebf104f5b8f6670c1c6e916 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 May 2023 12:33:26 +0200 Subject: [PATCH 007/473] Add forgotten file --- .../configs/merge_tree_too_many_parts.xml | 5 +++++ .../test_postgresql_replica_database_engine_2/test.py | 7 ++----- 2 files changed, 7 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml diff --git a/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml b/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml new file mode 100644 index 00000000000..4bc63453f55 --- /dev/null +++ b/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml @@ -0,0 +1,5 @@ + + + 5 + + diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3b5194e8806..07f356250ea 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -88,11 +88,8 @@ def setup_teardown(): def test_add_new_table_to_replication(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute("DROP TABLE IF EXISTS test_table") NUM_TABLES = 5 - - pg_manager.create_and_fill_postgres_tables_from_cursor(cursor, NUM_TABLES, 10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, 10000) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) @@ -105,7 +102,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_5" - pg_manager.create_and_fill_postgres_table_from_cursor(cursor, table_name) + pg_manager.create_and_fill_postgres_table(table_name) result = instance.query("SHOW CREATE DATABASE test_database") assert ( From ceecb1488af0fe4413053c87d9042b9d79602371 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 22 May 2023 17:29:52 +0800 Subject: [PATCH 008/473] add function arrayJaccardIndex --- src/Functions/array/arrayJaccardIndex.cpp | 161 ++++++++++++++++++ .../02737_arrayJaccardIndex.reference | 32 ++++ .../0_stateless/02737_arrayJaccardIndex.sql | 26 +++ 3 files changed, 219 insertions(+) create mode 100644 src/Functions/array/arrayJaccardIndex.cpp create mode 100644 tests/queries/0_stateless/02737_arrayJaccardIndex.reference create mode 100644 tests/queries/0_stateless/02737_arrayJaccardIndex.sql diff --git a/src/Functions/array/arrayJaccardIndex.cpp b/src/Functions/array/arrayJaccardIndex.cpp new file mode 100644 index 00000000000..8cce98ab64d --- /dev/null +++ b/src/Functions/array/arrayJaccardIndex.cpp @@ -0,0 +1,161 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; +} + +class FunctionArrayJaccardIndex : public IFunction +{ +public: + using ResultType = Float64; + static constexpr auto name = "arrayJaccardIndex"; + String getName() const override { return name; } + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } + explicit FunctionArrayJaccardIndex(ContextPtr context_) : context(context_) {} + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + DataTypes types; + for (size_t i = 0; i < 2; ++i) + { + const auto * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array, but it has type{}.", i + 1, getName(), arguments[i]->getName()); + } + return std::make_shared>(); + } + + template + static void vector(const ColumnArray::Offsets & intersect_offsets, const ColumnArray::Offsets & left_offsets, const ColumnArray::Offsets & right_offsets, PaddedPODArray & res) + { + size_t left_size; + size_t right_size; + for (size_t i = 0; i < res.size(); ++i) + { + if constexpr (is_const_left) + left_size = left_offsets[0]; + else + left_size = left_offsets[i] - left_offsets[i - 1]; + if constexpr (is_const_right) + right_size = right_offsets[0]; + else + right_size = right_offsets[i] - right_offsets[i - 1]; + + size_t intersect_size = intersect_offsets[i] - intersect_offsets[i - 1]; + res[i] = static_cast(intersect_size) / (left_size + right_size - intersect_size); + if (unlikely(isnan(res[i]))) + res[i] = 1; + } + } + + template + static void vectorWithEmptyIntersect(const ColumnArray::Offsets & left_offsets, const ColumnArray::Offsets & right_offsets, PaddedPODArray & res) + { + size_t left_size; + size_t right_size; + for (size_t i = 0; i < res.size(); ++i) + { + if constexpr (is_const_left) + left_size = left_offsets[0]; + else + left_size = left_offsets[i] - left_offsets[i - 1]; + if constexpr (is_const_right) + right_size = right_offsets[0]; + else + right_size = right_offsets[i] - right_offsets[i - 1]; + + res[i] = static_cast(left_size + right_size == 0); + } + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + bool is_const_left; + bool is_const_right; + const ColumnArray * left_array; + const ColumnArray * right_array; + + auto cast_array = [&](const ColumnWithTypeAndName & col) + { + const ColumnArray * res; + bool is_const = false; + if (typeid_cast(col.column.get())) + { + res = checkAndGetColumn(checkAndGetColumnConst(col.column.get())->getDataColumnPtr().get()); + is_const = true; + } + else if (!(res = checkAndGetColumn(col.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument for function {} must be array but it has type {}.", + col.column->getName(), getName()); + return std::make_pair(res, is_const); + }; + + std::tie(left_array, is_const_left) = cast_array(arguments[0]); + std::tie(right_array, is_const_right) = cast_array(arguments[1]); + + auto intersect_array = FunctionFactory::instance().get("arrayIntersect", context)->build(arguments); + ColumnWithTypeAndName intersect_column; + intersect_column.type = intersect_array->getResultType(); + intersect_column.column = intersect_array->execute(arguments, intersect_column.type, input_rows_count); + const auto * return_type_intersect = checkAndGetDataType(intersect_column.type.get()); + if (!return_type_intersect) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected return type for function arrayIntersect"); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(input_rows_count); + +#define EXECUTE_VECTOR(is_const_left, is_const_right) \ + if (typeid_cast(return_type_intersect->getNestedType().get())) \ + vectorWithEmptyIntersect(left_array->getOffsets(), right_array->getOffsets(), vec_res); \ + else \ + { \ + const ColumnArray * col_array = checkAndGetColumn(intersect_column.column.get()); \ + vector(col_array->getOffsets(), left_array->getOffsets(), right_array->getOffsets(), vec_res); \ + } + + if (!is_const_left && !is_const_right) + EXECUTE_VECTOR(false, false) + else if (!is_const_left && is_const_right) + EXECUTE_VECTOR(false, true) + else if (is_const_left && !is_const_right) + EXECUTE_VECTOR(true, false) + else + EXECUTE_VECTOR(true, true) + +#undef EXECUTE_VECTOR + + return col_res; + } + +private: + ContextPtr context; +}; + +REGISTER_FUNCTION(ArrayJaccardIndex) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02737_arrayJaccardIndex.reference b/tests/queries/0_stateless/02737_arrayJaccardIndex.reference new file mode 100644 index 00000000000..e6934bfe092 --- /dev/null +++ b/tests/queries/0_stateless/02737_arrayJaccardIndex.reference @@ -0,0 +1,32 @@ +0 +0.5 +1 +0.67 +1 +0 +0 +0 +1 +0 +0 +0 +0 +0.5 +1 +0.67 +0.5 +0.5 +0.5 +0.5 +1 +1 +1 +1 +1 +1 +1 +1 +0.33 +0.2 +1 +1 diff --git a/tests/queries/0_stateless/02737_arrayJaccardIndex.sql b/tests/queries/0_stateless/02737_arrayJaccardIndex.sql new file mode 100644 index 00000000000..c3f04ba0b10 --- /dev/null +++ b/tests/queries/0_stateless/02737_arrayJaccardIndex.sql @@ -0,0 +1,26 @@ +drop table if exists array_jaccard_index; + +create table array_jaccard_index (arr Array(UInt8)) engine=MergeTree partition by arr order by arr; + +insert into array_jaccard_index values ([1,2,3]); +insert into array_jaccard_index values ([1,2]); +insert into array_jaccard_index values ([1]); +insert into array_jaccard_index values ([]); + +select round(arrayJaccardIndex(arr, [1,2]), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex(arr, []), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([], arr), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([1,2], arr), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([1,2], [1,2,3,4]), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([], []), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex(arr, arr), 2) from array_jaccard_index order by arr; + +drop table if exists array_jaccard_index; + +select round(arrayJaccardIndex(['a'], ['a', 'aa', 'aaa']), 2); + +select round(arrayJaccardIndex([1, 1.1, 2.2], [2.2, 3.3, 444]), 2); + +select round(arrayJaccardIndex([], []), 2); + +select round(arrayJaccardIndex([toUInt16(1)], [toUInt32(1)]), 2); From 656d6abb2ee2d221df01367bcb53465289e4981c Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 22 May 2023 22:49:02 +0800 Subject: [PATCH 009/473] fix style --- src/Functions/array/arrayJaccardIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayJaccardIndex.cpp b/src/Functions/array/arrayJaccardIndex.cpp index 8cce98ab64d..c1ec8b53d25 100644 --- a/src/Functions/array/arrayJaccardIndex.cpp +++ b/src/Functions/array/arrayJaccardIndex.cpp @@ -100,7 +100,7 @@ public: { const ColumnArray * res; bool is_const = false; - if (typeid_cast(col.column.get())) + if (typeid_cast(col.column.get())) { res = checkAndGetColumn(checkAndGetColumnConst(col.column.get())->getDataColumnPtr().get()); is_const = true; From b0b9f2a037918b8f745df952a9491b97de6fdada Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Tue, 23 May 2023 10:39:23 +0800 Subject: [PATCH 010/473] fix test --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 79a6ad1fa2d..5ef83a57ecf 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -112,6 +112,7 @@ arrayFirstIndex arrayFirstOrNull arrayFlatten arrayIntersect +arrayJaccardIndex arrayJoin arrayLast arrayLastIndex From 1a361ef3060a4d271e6e27bb816df1d18cffaa02 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 5 Jun 2023 03:21:43 +0200 Subject: [PATCH 011/473] works for file --- src/Storages/StorageFile.cpp | 103 +++++++++++++++++++++++++++++++---- 1 file changed, 93 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 647f9511052..53da509d383 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -90,6 +90,57 @@ namespace ErrorCodes namespace { +/// Forward-declare to use in listFilesWithFoldedRegexpMatchingImpl() +void listFilesWithRegexpMatchingImpl( + const std::string & path_for_ls, + const std::string & for_match, + size_t & total_bytes_to_read, + std::vector & result, + bool recursive = false); + +/// When `{...}` has any `/`s, it must be processed in a different way +void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const std::string & processed_suffix, + const std::string & suffix_with_globs, + const std::string & glob, re2::RE2 & matcher, size_t & total_bytes_to_read, + const size_t max_depth, const size_t next_slash_after_glob_pos, + std::vector & result) +{ + /// We don't need to go all the way in every directory if max_depth is reached + /// as it is upper limit of depth by simply counting `/`s in curly braces + if (!max_depth) + return; + + const fs::directory_iterator end; + for (fs::directory_iterator it(start_dir + processed_suffix); it != end; ++it) + { + const std::string full_path = it->path().string(); + const size_t last_slash = full_path.rfind('/'); + const String dir_or_file_name = full_path.substr(last_slash); + + if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) + { + if (next_slash_after_glob_pos == std::string::npos) + { + total_bytes_to_read += it->file_size(); + result.push_back(it->path().string()); + } + else + { + listFilesWithRegexpMatchingImpl(fs::path(full_path).append(processed_suffix).append(it->path().string()) / "" , + suffix_with_globs.substr(next_slash_after_glob_pos), + total_bytes_to_read, result); + } + } + else if (it->is_directory()) + { + listFilesWithFoldedRegexpMatchingImpl(start_dir, processed_suffix + dir_or_file_name, suffix_with_globs, + glob, matcher, total_bytes_to_read, max_depth - 1, + next_slash_after_glob_pos, result); + } + + } +} + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageHDFS. */ @@ -98,15 +149,40 @@ void listFilesWithRegexpMatchingImpl( const std::string & for_match, size_t & total_bytes_to_read, std::vector & result, - bool recursive = false) + bool recursive) { - const size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob_pos = for_match.find_first_of("*?{"); + const bool has_glob = first_glob_pos != std::string::npos; - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const size_t next_slash = suffix_with_globs.find('/', 1); - const std::string current_glob = suffix_with_globs.substr(0, next_slash); + /// slashes_in_glob counter is a upper-bound estimate of recursion depth + /// needed to process complex cases when `/` is included into glob, e.g. /pa{th1/a,th2/b}.csv + size_t slashes_in_glob = 0; + const size_t next_slash_after_glob_pos = [&](){ + if (!has_glob) + return suffix_with_globs.find('/', 1); + + size_t in_curly = 0; + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + if (*it == '{') + ++in_curly; + else if (*it == '/') + { + if (in_curly) + ++slashes_in_glob; + else + return size_t(std::distance(suffix_with_globs.begin(), it)); + } + else if (*it == '}') + --in_curly; + } + return std::string::npos; + }(); + + const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); + auto regexp = makeRegexpPatternFromGlobs(current_glob); re2::RE2 matcher(regexp); @@ -123,13 +199,22 @@ void listFilesWithRegexpMatchingImpl( if (!fs::exists(prefix_without_globs)) return; + const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; + + if (slashes_in_glob) + { + listFilesWithFoldedRegexpMatchingImpl(prefix_without_globs, "", suffix_with_globs, + current_glob, matcher, total_bytes_to_read, slashes_in_glob, + next_slash_after_glob_pos, result); + return; + } + const fs::directory_iterator end; for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; /// Condition is_directory means what kind of path is it in current iteration of ls if (!it->is_directory() && !looking_for_directory) @@ -145,14 +230,12 @@ void listFilesWithRegexpMatchingImpl( if (recursive) { listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "" , - looking_for_directory ? suffix_with_globs.substr(next_slash) : current_glob , + looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob , total_bytes_to_read, result, recursive); } else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) - { /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read, result); - } + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); } } } From ece53808519ce6e42bbff2f2f3ac7d9ffccabe63 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 7 Jun 2023 15:48:38 +0200 Subject: [PATCH 012/473] add simple stateless for file() --- .../02771_complex_globs_in_storage_file_path.reference | 4 ++++ .../02771_complex_globs_in_storage_file_path.sql | 8 ++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference create mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference new file mode 100644 index 00000000000..e1b420ecf37 --- /dev/null +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference @@ -0,0 +1,4 @@ +This is file data1 data1.csv +This is file data2 data2.csv +This is file data1 data1.csv +This is file data2 data2.csv diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql new file mode 100644 index 00000000000..c579c8d8698 --- /dev/null +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql @@ -0,0 +1,8 @@ +-- Tags: no-replicated-database, no-parallel + +INSERT INTO TABLE FUNCTION file('02771/dir1/subdir11/data1.csv', 'CSV', 's String') SELECT 'This is file data1' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's String') SELECT 'This is file data2' SETTINGS engine_file_truncate_on_insert=1; + +SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); +SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV); From bfdb18619c3b31044eba28f15cd9a0d7141b790e Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 7 Jun 2023 19:25:33 +0200 Subject: [PATCH 013/473] small docs udpate --- docs/en/sql-reference/table-functions/file.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 28c2dc9f1f3..09e3df7cc6a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -134,7 +134,7 @@ Multiple path components can have globs. For being processed file must exist and - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 94bc734a8fb..f0fea630ad2 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). From 2c97a94892358f7e6b4a7aa4b25c4d46dd0f0fc0 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 11 Jun 2023 01:50:17 +0200 Subject: [PATCH 014/473] fix hdfs + style update --- src/Storages/HDFS/StorageHDFS.cpp | 115 ++++++++++++++++++++++++++++-- src/Storages/StorageFile.cpp | 20 +++--- 2 files changed, 120 insertions(+), 15 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19c0840149b..379ee395939 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -63,23 +63,124 @@ namespace ErrorCodes } namespace { + Strings LSWithRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + const String & for_match, + std::unordered_map * last_mod_times); + + /// When `{...}` has any `/`s, it must be processed in a different way + Strings LSWithFoldedRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + std::unordered_map * last_mod_times, + const String & processed_suffix, + const String & suffix_with_globs, + const String & current_glob, + re2::RE2 & matcher, + const size_t max_depth, + const size_t next_slash_after_glob_pos) + { + /// We don't need to go all the way in every directory if max_depth is reached + /// as it is upper limit of depth by simply counting `/`s in curly braces + if (!max_depth) + return {}; + + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); + if (ls.file_info == nullptr && errno != ENOENT) // NOLINT + { + // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. + throw Exception( + ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); + } + + Strings result; + if (!ls.file_info && ls.length > 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); + + for (int i = 0; i < ls.length; ++i) + { + const String full_path = String(ls.file_info[i].mName); + const size_t last_slash = full_path.rfind('/'); + const String dir_or_file_name = full_path.substr(last_slash); + const bool is_directory = ls.file_info[i].mKind == 'D'; + + if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) + { + if (next_slash_after_glob_pos == std::string::npos) + { + result.push_back(String(ls.file_info[i].mName)); + if (last_mod_times) + (*last_mod_times)[result.back()] = ls.file_info[i].mLastMod; + } + else + { + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "" , fs, + suffix_with_globs.substr(next_slash_after_glob_pos), last_mod_times); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + else if (is_directory) + { + Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path).append(processed_suffix), + fs, last_mod_times, processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, + max_depth - 1, next_slash_after_glob_pos); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + return result; + } + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageFile. */ - Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times) + Strings LSWithRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + const String & for_match, + std::unordered_map * last_mod_times) { - const size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob_pos = for_match.find_first_of("*?{"); + const bool has_glob = first_glob_pos != std::string::npos; - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - const size_t next_slash = suffix_with_globs.find('/', 1); - re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); + size_t slashes_in_glob = 0; + const size_t next_slash_after_glob_pos = [&](){ + if (!has_glob) + return suffix_with_globs.find('/', 1); + + size_t in_curly = 0; + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + if (*it == '{') + ++in_curly; + else if (*it == '/') + { + if (in_curly) + ++slashes_in_glob; + else + return size_t(std::distance(suffix_with_globs.begin(), it)); + } + else if (*it == '}') + --in_curly; + } + return std::string::npos; + }(); + + const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); + + re2::RE2 matcher(makeRegexpPatternFromGlobs(current_glob)); if (!matcher.ok()) throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); + if (slashes_in_glob) + { + return LSWithFoldedRegexpMatching(fs::path(prefix_without_globs), fs, last_mod_times, + "", suffix_with_globs, current_glob, matcher, + slashes_in_glob, next_slash_after_glob_pos); + } + HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); if (ls.file_info == nullptr && errno != ENOENT) // NOLINT @@ -96,7 +197,7 @@ namespace const String full_path = String(ls.file_info[i].mName); const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; + const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; const bool is_directory = ls.file_info[i].mKind == 'D'; /// Condition with type of current file_info means what kind of path is it in current iteration of ls if (!is_directory && !looking_for_directory) @@ -112,7 +213,7 @@ namespace { if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash), last_mod_times); + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash_after_glob_pos), last_mod_times); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 53da509d383..74303b16ee9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -99,10 +99,14 @@ void listFilesWithRegexpMatchingImpl( bool recursive = false); /// When `{...}` has any `/`s, it must be processed in a different way -void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const std::string & processed_suffix, +void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, + const std::string & processed_suffix, const std::string & suffix_with_globs, - const std::string & glob, re2::RE2 & matcher, size_t & total_bytes_to_read, - const size_t max_depth, const size_t next_slash_after_glob_pos, + const std::string & current_glob, + re2::RE2 & matcher, + size_t & total_bytes_to_read, + const size_t max_depth, + const size_t next_slash_after_glob_pos, std::vector & result) { /// We don't need to go all the way in every directory if max_depth is reached @@ -111,7 +115,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const return; const fs::directory_iterator end; - for (fs::directory_iterator it(start_dir + processed_suffix); it != end; ++it) + for (fs::directory_iterator it(path_for_ls); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); @@ -133,9 +137,9 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const } else if (it->is_directory()) { - listFilesWithFoldedRegexpMatchingImpl(start_dir, processed_suffix + dir_or_file_name, suffix_with_globs, - glob, matcher, total_bytes_to_read, max_depth - 1, - next_slash_after_glob_pos, result); + listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path).append(processed_suffix), processed_suffix + dir_or_file_name, + suffix_with_globs, current_glob, matcher, + total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, result); } } @@ -203,7 +207,7 @@ void listFilesWithRegexpMatchingImpl( if (slashes_in_glob) { - listFilesWithFoldedRegexpMatchingImpl(prefix_without_globs, "", suffix_with_globs, + listFilesWithFoldedRegexpMatchingImpl(fs::path(prefix_without_globs), "", suffix_with_globs, current_glob, matcher, total_bytes_to_read, slashes_in_glob, next_slash_after_glob_pos, result); return; From aad7712b18fa358f60c59565d3914fafc2a3f759 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 11 Jun 2023 02:09:05 +0200 Subject: [PATCH 015/473] add existing test --- tests/integration/test_storage_hdfs/test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index edf5344e887..0b18c0180cc 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -85,6 +85,15 @@ def test_read_write_storage_with_globs(started_cluster): assert "in readonly mode" in str(ex) +def test_storage_with_multidirectory_glob(started_cluster): + hdfs_api = started_cluster.hdfs_api + for i in ["1", "2"]: + hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") + assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" + + assert node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") == f"\File1\t11\nFile2\t22\n" + + def test_read_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api From 427c5cb1bafef9b52011f9d77e725b7a5ba85553 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sun, 11 Jun 2023 18:42:10 +0200 Subject: [PATCH 016/473] fix integration test --- tests/integration/test_storage_hdfs/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 0b18c0180cc..dde3dd257b1 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -91,7 +91,8 @@ def test_storage_with_multidirectory_glob(started_cluster): hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" - assert node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") == f"\File1\t11\nFile2\t22\n" + r = node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") + assert (r == f"File1\t11\nFile2\t22\n") or (r == f"File2\t22\nFile1\t11\n") def test_read_write_table(started_cluster): From eb9cdbcf7d5d1cdcc3b45936b4045dd8eda8f818 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 11:41:36 +0200 Subject: [PATCH 017/473] fix File test being flaky --- src/Storages/StorageFile.cpp | 15 +++++++++++---- .../02771_complex_globs_in_storage_file_path.sql | 4 +++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 74303b16ee9..c7a57b7d4c9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -109,8 +109,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t next_slash_after_glob_pos, std::vector & result) { - /// We don't need to go all the way in every directory if max_depth is reached - /// as it is upper limit of depth by simply counting `/`s in curly braces + /* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * LSWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageHDFS.cpp has the same logic. + */ if (!max_depth) return; @@ -121,6 +127,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t last_slash = full_path.rfind('/'); const String dir_or_file_name = full_path.substr(last_slash); + std::cerr << "\nprocessing file (full_path): " << full_path << "\n"; if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) { if (next_slash_after_glob_pos == std::string::npos) @@ -130,14 +137,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, } else { - listFilesWithRegexpMatchingImpl(fs::path(full_path).append(processed_suffix).append(it->path().string()) / "" , + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "" , suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); } } else if (it->is_directory()) { - listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path).append(processed_suffix), processed_suffix + dir_or_file_name, + listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path), processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, result); } diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql index c579c8d8698..41d7d6fcc3f 100644 --- a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql @@ -5,4 +5,6 @@ INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's Strin SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); -SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV); + +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv'; +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv'; From 7d7bd5bb66fbf1e0d3a1e35a0fe74231d6e5e7ed Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 11:50:23 +0200 Subject: [PATCH 018/473] update comment describing workflow --- src/Storages/HDFS/StorageHDFS.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 379ee395939..bd50b66ede5 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -63,12 +63,20 @@ namespace ErrorCodes } namespace { + /// Forward-declared to use in LSWithFoldedRegexpMatching w/o circular dependency. Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times); - /// When `{...}` has any `/`s, it must be processed in a different way + /* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * LSWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageFile.cpp has the same logic. + */ Strings LSWithFoldedRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, std::unordered_map * last_mod_times, From 3e6d393e17b0913d2664838a98a8f946bc8b7695 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 12:06:21 +0200 Subject: [PATCH 019/473] remove debug cerr --- src/Storages/StorageFile.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c7a57b7d4c9..93228cf4d39 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -127,7 +127,6 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t last_slash = full_path.rfind('/'); const String dir_or_file_name = full_path.substr(last_slash); - std::cerr << "\nprocessing file (full_path): " << full_path << "\n"; if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) { if (next_slash_after_glob_pos == std::string::npos) From cd1a3916a6ea755b24b475983f6f67447cebdd6a Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 12 Jun 2023 13:28:17 +0200 Subject: [PATCH 020/473] cleanup HDFS --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index bd50b66ede5..7e836c028a0 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -129,7 +129,7 @@ namespace } else if (is_directory) { - Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path).append(processed_suffix), + Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path), fs, last_mod_times, processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, max_depth - 1, next_slash_after_glob_pos); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); From 1d8013074de7eb5a35460fef94bc82b8872531c3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 13 Jun 2023 18:47:02 +0200 Subject: [PATCH 021/473] fix style & black --- src/Storages/HDFS/StorageHDFS.cpp | 6 ++++-- src/Storages/StorageFile.cpp | 6 ++++-- tests/integration/test_storage_hdfs/test.py | 13 ++++++++++--- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 7e836c028a0..2503b0e60f2 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -154,12 +154,14 @@ namespace const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' size_t slashes_in_glob = 0; - const size_t next_slash_after_glob_pos = [&](){ + const size_t next_slash_after_glob_pos = [&]() + { if (!has_glob) return suffix_with_globs.find('/', 1); size_t in_curly = 0; - for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) + { if (*it == '{') ++in_curly; else if (*it == '/') diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 93228cf4d39..dc7cfe0de29 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -170,12 +170,14 @@ void listFilesWithRegexpMatchingImpl( /// slashes_in_glob counter is a upper-bound estimate of recursion depth /// needed to process complex cases when `/` is included into glob, e.g. /pa{th1/a,th2/b}.csv size_t slashes_in_glob = 0; - const size_t next_slash_after_glob_pos = [&](){ + const size_t next_slash_after_glob_pos = [&]() + { if (!has_glob) return suffix_with_globs.find('/', 1); size_t in_curly = 0; - for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) + { if (*it == '{') ++in_curly; else if (*it == '/') diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index dde3dd257b1..789e8e11035 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -88,10 +88,17 @@ def test_read_write_storage_with_globs(started_cluster): def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api for i in ["1", "2"]: - hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") - assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" + hdfs_api.write_data( + f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n" + ) + assert ( + hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") + == f"File{i}\t{i}{i}\n" + ) - r = node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") + r = node1.query( + "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)" + ) assert (r == f"File1\t11\nFile2\t22\n") or (r == f"File2\t22\nFile1\t11\n") From 02cf8a1bade3898196d7a760cb75ae10af3f2ecf Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 15 Jun 2023 14:09:38 +0300 Subject: [PATCH 022/473] Update comment StorageFile.cpp --- src/Storages/StorageFile.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 41ddd6b5780..4fbea89baa2 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -100,7 +100,14 @@ void listFilesWithRegexpMatchingImpl( std::vector & result, bool recursive = false); -/// When `{...}` has any `/`s, it must be processed in a different way +/* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by listFilesWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * listFilesWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageHDFS.cpp has the same logic. +*/ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const std::string & processed_suffix, const std::string & suffix_with_globs, @@ -111,14 +118,6 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t next_slash_after_glob_pos, std::vector & result) { - /* - * When `{...}` has any `/`s, it must be processed in a different way: - * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, - * LSWithFoldedRegexpMatching is in charge from now on. - * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. - * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. - * StorageHDFS.cpp has the same logic. - */ if (!max_depth) return; From 9a25958be831d6068597703cef97238ab76ee325 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 15 Jun 2023 13:49:49 +0000 Subject: [PATCH 023/473] Add HTTP header filtering --- programs/server/Server.cpp | 2 + programs/server/config.xml | 8 +++ src/Common/HTTPHeaderFilter.cpp | 56 +++++++++++++++++++ src/Common/HTTPHeaderFilter.h | 27 +++++++++ src/Interpreters/Context.cpp | 16 +++++- src/Interpreters/Context.h | 5 ++ src/Storages/StorageS3.cpp | 1 + src/Storages/StorageS3Cluster.cpp | 2 + src/Storages/StorageURL.cpp | 1 + src/Storages/StorageURLCluster.cpp | 1 + tests/config/config.d/forbidden_headers.xml | 6 ++ tests/config/install.sh | 1 + .../02752_forbidden_headers.reference | 0 .../0_stateless/02752_forbidden_headers.sql | 18 ++++++ 14 files changed, 142 insertions(+), 2 deletions(-) create mode 100644 src/Common/HTTPHeaderFilter.cpp create mode 100644 src/Common/HTTPHeaderFilter.h create mode 100644 tests/config/config.d/forbidden_headers.xml create mode 100644 tests/queries/0_stateless/02752_forbidden_headers.reference create mode 100644 tests/queries/0_stateless/02752_forbidden_headers.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d0fc8aca5e8..6490eb6e3f5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -887,6 +887,7 @@ try #endif global_context->setRemoteHostFilter(config()); + global_context->setHTTPHeaderFilter(config()); std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); fs::path path = path_str; @@ -1184,6 +1185,7 @@ try } global_context->setRemoteHostFilter(*config); + global_context->setHTTPHeaderFilter(*config); global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop); global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop); diff --git a/programs/server/config.xml b/programs/server/config.xml index d18b4cb2ac9..f5013104630 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1006,6 +1006,14 @@ --> + + + s3 s3_disk/ @@ -10,7 +9,6 @@ clickhouse 20000 - cache s3_disk @@ -88,13 +86,6 @@ - - -
- s3_cache_small -
-
-
From 7c3c48c8c44a1c53902dd24d540e25e2634a986b Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 5 Jul 2023 11:20:03 +0000 Subject: [PATCH 103/473] Fix use_structure_from_insertion_table_in_table_functions does not work for materialized and alias columns --- src/Interpreters/Context.cpp | 6 +++++- .../0_stateless/02811_insert_schema_inference.reference | 0 .../0_stateless/02811_insert_schema_inference.sql | 9 +++++++++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02811_insert_schema_inference.reference create mode 100644 tests/queries/0_stateless/02811_insert_schema_inference.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7482450d529..a0abab349b3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1519,7 +1519,11 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const uint64_t use_structure_from_insertion_table_in_table_functions = getSettingsRef().use_structure_from_insertion_table_in_table_functions; if (use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insert_structure = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); + const auto & insert_structure = DatabaseCatalog::instance() + .getTable(getInsertionTable(), shared_from_this()) + ->getInMemoryMetadataPtr() + ->getColumns() + .getInsertable(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; diff --git a/tests/queries/0_stateless/02811_insert_schema_inference.reference b/tests/queries/0_stateless/02811_insert_schema_inference.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02811_insert_schema_inference.sql b/tests/queries/0_stateless/02811_insert_schema_inference.sql new file mode 100644 index 00000000000..9de710047f7 --- /dev/null +++ b/tests/queries/0_stateless/02811_insert_schema_inference.sql @@ -0,0 +1,9 @@ +drop table if exists test; +create table test +( + n1 UInt32, + n2 UInt32 alias murmurHash3_32(n1), + n3 UInt32 materialized n2 + 1 +)engine=MergeTree order by n1; +insert into test select * from generateRandom() limit 10; +drop table test; From 665b6d43342f418a9c32bc31d1a969b53766fc96 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 5 Jul 2023 14:50:15 +0200 Subject: [PATCH 104/473] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 526e674a154..636b7a9747d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -761,6 +761,7 @@ Rollup RowBinary RowBinaryWithNames RowBinaryWithNamesAndTypes +RowBinaryWithDefaults Runtime SATA SELECTs @@ -2118,6 +2119,7 @@ rowNumberInBlock rowbinary rowbinarywithnames rowbinarywithnamesandtypes +rowbinarywithdefaults rsync rsyslog runnable From 2ab6c599a234d31c2f59e1aaa35298c1274390b1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 Jul 2023 23:31:44 +0200 Subject: [PATCH 105/473] Fix tests --- .../0_stateless/02240_filesystem_query_cache.sql | 15 +++++++++++++++ .../0_stateless/02286_drop_filesystem_cache.sh | 5 ----- .../0_stateless/02344_describe_cache.reference | 2 +- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 760ec1baa87..02cf54b0caa 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -9,6 +9,21 @@ SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; SYSTEM DROP FILESYSTEM CACHE; +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations= 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh index 091bca10bcf..1e1841862e9 100755 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh @@ -67,9 +67,4 @@ for STORAGE_POLICY in 's3_cache' 'local_cache'; do ON data_paths.cache_path = caches.cache_path" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_022862" - - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_022862 (key UInt32, value String) - Engine=MergeTree() - ORDER BY key - SETTINGS storage_policy='${STORAGE_POLICY}_2', min_bytes_for_wide_part = 10485760" done diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 5c3d47d87f6..da84cdabf79 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1 +1 @@ -134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0 From c178a362c573f7212c8f9986f78e78b209713bee Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 6 Jul 2023 02:30:37 +0000 Subject: [PATCH 106/473] Fix for new analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 163092f1b7f..34286c266c9 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6238,7 +6238,11 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, const auto & insertion_table = scope_context->getInsertionTable(); if (!insertion_table.empty()) { - const auto & insert_structure = DatabaseCatalog::instance().getTable(insertion_table, scope_context)->getInMemoryMetadataPtr()->getColumns(); + const auto & insert_structure = DatabaseCatalog::instance() + .getTable(insertion_table, scope_context) + ->getInMemoryMetadataPtr() + ->getColumns() + .getInsertable(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; From 546f12dc85fdbbcf3396767917bd9dbbf8522c41 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 05:05:27 +0000 Subject: [PATCH 107/473] Fix inserts to MongoDB tables --- src/Storages/StorageMongoDB.cpp | 60 ++++++++++++++++++- .../integration/test_storage_mongodb/test.py | 6 ++ 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 3287e3272e3..45b8aceb058 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -19,6 +19,8 @@ #include #include +#include + namespace DB { @@ -127,9 +129,7 @@ public: for (const auto j : collections::range(0, num_cols)) { - WriteBufferFromOwnString ostr; - data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{}); - document->add(data_names[j], ostr.str()); + insertValueIntoMongoDB(*document, data_names[j], *data_types[j], *columns[j], i); } documents.push_back(std::move(document)); @@ -151,6 +151,60 @@ public: } private: + + void insertValueIntoMongoDB( + Poco::MongoDB::Document & document, + const std::string & name, + const IDataType & data_type, + const IColumn & column, + size_t idx) + { + WhichDataType which(data_type); + + if (which.isArray()) + { + const ColumnArray & column_array = assert_cast(column); + const ColumnArray::Offsets & offsets = column_array.getOffsets(); + + size_t offset = offsets[idx - 1]; + size_t next_offset = offsets[idx]; + + const IColumn & nested_column = column_array.getData(); + + const auto * array_type = assert_cast(&data_type); + const DataTypePtr & nested_type = array_type->getNestedType(); + + Poco::MongoDB::Array::Ptr array = new Poco::MongoDB::Array(); + for (size_t i = 0; i + offset < next_offset; ++i) + { + insertValueIntoMongoDB(*array, Poco::NumberFormatter::format(i), *nested_type, nested_column, i + offset); + } + + document.add(name, array); + return; + } + + /// MongoDB does not support UInt64 type, so just cast it to Int64 + if (which.isNativeUInt()) + document.add(name, static_cast(column.getUInt(idx))); + else if (which.isNativeInt()) + document.add(name, static_cast(column.getInt(idx))); + else if (which.isFloat32()) + document.add(name, static_cast(column.getFloat32(idx))); + else if (which.isFloat64()) + document.add(name, static_cast(column.getFloat64(idx))); + else if (which.isDate()) + document.add(name, Poco::Timestamp(DateLUT::instance().fromDayNum(DayNum(column.getUInt(idx))) * 1000000)); + else if (which.isDateTime()) + document.add(name, Poco::Timestamp(column.getUInt(idx) * 1000000)); + else + { + WriteBufferFromOwnString ostr; + data_type.getDefaultSerialization()->serializeText(column, idx, ostr, FormatSettings{}); + document.add(name, ostr.str()); + } + } + String collection_name; String db_name; StorageMetadataPtr metadata_snapshot; diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 6ce71fb91fa..0abaa7a8214 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -244,6 +244,12 @@ def test_arrays(started_cluster): == "[]\n" ) + # Test INSERT SELECT + node.query("INSERT INTO arrays_mongo_table SELECT * FROM arrays_mongo_table") + + assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "200\n" + assert node.query("SELECT COUNT(DISTINCT *) FROM arrays_mongo_table") == "100\n" + node.query("DROP TABLE arrays_mongo_table") arrays_mongo_table.drop() From d86ceef663cd0d3fcd8532ae63539e85bc4b210b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:14:48 +0800 Subject: [PATCH 108/473] Implement log file names rendering --- src/Loggers/Loggers.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 0c3a7bd615d..1e169190ca4 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,6 +34,16 @@ static std::string createDirectory(const std::string & file) return path; } +static std::string renderFileNameTemplate(time_t now, const std::string & file_path) +{ + fs::path path{file_path}; + std::tm buf; + localtime_r(&now, &buf); + std::stringstream ss; + ss << std::put_time(&buf, file_path.c_str()); + return path.replace_filename(ss.str()); +} + #ifndef WITHOUT_TEXT_LOG void Loggers::setTextLog(std::shared_ptr log, int max_priority) { @@ -68,9 +78,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// The maximum (the most verbose) of those will be used as default for Poco loggers int max_log_level = 0; - const auto log_path = config.getString("logger.log", ""); - if (!log_path.empty()) + time_t now = std::time({}); + + const auto log_path_prop = config.getString("logger.log", ""); + if (!log_path_prop.empty()) { + const auto log_path = renderFileNameTemplate(now, log_path_prop); createDirectory(log_path); std::string ext; @@ -109,9 +122,10 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split->addChannel(log, "log"); } - const auto errorlog_path = config.getString("logger.errorlog", ""); - if (!errorlog_path.empty()) + const auto errorlog_path_prop = config.getString("logger.errorlog", ""); + if (!errorlog_path_prop.empty()) { + const auto errorlog_path = renderFileNameTemplate(now, errorlog_path_prop); createDirectory(errorlog_path); // NOTE: we don't use notice & critical in the code, so in practice error log collects fatal & error & warning. From 479efaa79acd23e72fb06413fd84d4b7091bd019 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:16:18 +0800 Subject: [PATCH 109/473] Add clickhouse_log_file and clickhouse_error_log_file args to add_instance() --- tests/integration/helpers/cluster.py | 35 +++++++++++++++++----------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 21398790be3..5b583b865de 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -64,6 +64,13 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" +CLICKHOUSE_START_COMMAND = ( + "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file}" +) + +CLICKHOUSE_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.log" + +CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.log" # to create docker-compose env file def _create_env_file(path, variables): @@ -1497,6 +1504,8 @@ class ClickHouseCluster: with_postgres=False, with_postgres_cluster=False, with_postgresql_java_client=False, + clickhouse_log_file=CLICKHOUSE_LOG_FILE, + clickhouse_error_log_file=CLICKHOUSE_ERROR_LOG_FILE, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, @@ -1563,6 +1572,13 @@ class ClickHouseCluster: "LLVM_PROFILE_FILE" ] = "/var/lib/clickhouse/server_%h_%p_%m.profraw" + clickhouse_start_command = CLICKHOUSE_START_COMMAND + if clickhouse_log_file: + clickhouse_start_command += " --log-file=" + clickhouse_log_file + if clickhouse_error_log_file: + clickhouse_start_command += " --errorlog-file=" + clickhouse_error_log_file + logging.debug(f"clickhouse_start_command: {clickhouse_start_command}") + instance = ClickHouseInstance( cluster=self, base_path=self.base_dir, @@ -1592,10 +1608,10 @@ class ClickHouseCluster: with_redis=with_redis, with_minio=with_minio, with_azurite=with_azurite, - with_cassandra=with_cassandra, with_jdbc_bridge=with_jdbc_bridge, with_hive=with_hive, with_coredns=with_coredns, + with_cassandra=with_cassandra, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -1604,6 +1620,10 @@ class ClickHouseCluster: with_postgres=with_postgres, with_postgres_cluster=with_postgres_cluster, with_postgresql_java_client=with_postgresql_java_client, + clickhouse_start_command=clickhouse_start_command, + main_config_name=main_config_name, + users_config_name=users_config_name, + copy_common_configs=copy_common_configs, hostname=hostname, env_variables=env_variables, image=image, @@ -1612,9 +1632,6 @@ class ClickHouseCluster: ipv4_address=ipv4_address, ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, - main_config_name=main_config_name, - users_config_name=users_config_name, - copy_common_configs=copy_common_configs, external_dirs=external_dirs, tmpfs=tmpfs or [], config_root_name=config_root_name, @@ -3046,16 +3063,6 @@ class ClickHouseCluster: subprocess_check_call(self.base_zookeeper_cmd + ["start", n]) -CLICKHOUSE_START_COMMAND = ( - "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file}" - " --log-file=/var/log/clickhouse-server/clickhouse-server.log " - " --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" -) - -CLICKHOUSE_STAY_ALIVE_COMMAND = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( - CLICKHOUSE_START_COMMAND -) - DOCKER_COMPOSE_TEMPLATE = """ version: '2.3' services: From fef71ab0b8759f7a659c4bb8c1be03a89df92f79 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:16:46 +0800 Subject: [PATCH 110/473] Add inegration test --- .../__init__.py | 58 +++++++++++++++++++ .../configs/config-file-template.xml | 6 ++ .../test.py | 0 3 files changed, 64 insertions(+) create mode 100644 tests/integration/test_render_log_file_name_templates/__init__.py create mode 100644 tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml create mode 100644 tests/integration/test_render_log_file_name_templates/test.py diff --git a/tests/integration/test_render_log_file_name_templates/__init__.py b/tests/integration/test_render_log_file_name_templates/__init__.py new file mode 100644 index 00000000000..9fa87056d2c --- /dev/null +++ b/tests/integration/test_render_log_file_name_templates/__init__.py @@ -0,0 +1,58 @@ +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from datetime import datetime + + +log_dir = "/var/log/clickhouse-server/" +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.add_instance( + "file-names-from-config", + main_configs=["configs/config-file-template.xml"], + clickhouse_log_file=None, + clickhouse_error_log_file=None, + ) + cluster.add_instance( + "file-names-from-params", + clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", + clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", + ) + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_file_names(started_cluster): + now = datetime.now() + log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" + ) + err_log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" + ) + logging.debug(f"log_file {log_file} err_log_file {err_log_file}") + + for name, instance in started_cluster.instances.items(): + files = instance.exec_in_container( + ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True + ) + + logging.debug(f"check instance '{name}': {log_dir} contains: {files}") + + assert ( + instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) + == log_file + "\n" + ) + + assert ( + instance.exec_in_container( + ["bash", "-c", f"ls {err_log_file}"], nothrow=True + ) + == err_log_file + "\n" + ) diff --git a/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml b/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml new file mode 100644 index 00000000000..ba408eb9823 --- /dev/null +++ b/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml @@ -0,0 +1,6 @@ + + + /var/log/clickhouse-server/clickhouse-server-%Y-%m.log + /var/log/clickhouse-server/clickhouse-server-%Y-%m.err.log + + diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py new file mode 100644 index 00000000000..e69de29bb2d From b9fffacc653fb9175af03cbb8f53766b0272ddbc Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 06:31:09 +0000 Subject: [PATCH 111/473] Fix build --- src/Storages/StorageMongoDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 45b8aceb058..21543541f36 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -186,9 +186,9 @@ private: /// MongoDB does not support UInt64 type, so just cast it to Int64 if (which.isNativeUInt()) - document.add(name, static_cast(column.getUInt(idx))); + document.add(name, static_cast(column.getUInt(idx))); else if (which.isNativeInt()) - document.add(name, static_cast(column.getInt(idx))); + document.add(name, static_cast(column.getInt(idx))); else if (which.isFloat32()) document.add(name, static_cast(column.getFloat32(idx))); else if (which.isFloat64()) From 9a295eca46fea2c88d1c1767fc4625b31c999572 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jul 2023 14:28:50 +0000 Subject: [PATCH 112/473] Incorporate review feedback --- docs/en/sql-reference/statements/show.md | 24 ++--- .../InterpreterShowIndexesQuery.cpp | 4 +- .../0_stateless/02724_show_indexes.reference | 88 +++++++++---------- 3 files changed, 58 insertions(+), 58 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 1a1e4dbd2c7..1c399d2072b 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -289,18 +289,18 @@ The statement produces a result table with the following structure: - table - The name of the table. (String) - non_unique - Always `1` as ClickHouse does not support uniqueness constraints. (UInt8) - key_name - The name of the index, `PRIMARY` if the index is a primary key index. (String) -- column_name - For a primary key index, the name of the column. For a data skipping index: '' (empty string), see field "expression". (String) - seq_in_index - For a primary key index, the position of the column starting from `1`. For a data skipping index: always `1`. (UInt8) +- column_name - For a primary key index, the name of the column. For a data skipping index: `''` (empty string), see field "expression". (String) - collation - The sorting of the column in the index: `A` if ascending, `D` if descending, `NULL` if unsorted. (Nullable(String)) - cardinality - An estimation of the index cardinality (number of unique values in the index). Currently always 0. (UInt64) - sub_part - Always `NULL` because ClickHouse does not support index prefixes like MySQL. (Nullable(String)) - packed - Always `NULL` because ClickHouse does not support packed indexes (like MySQL). (Nullable(String)) - null - Currently unused - index_type - The index type, e.g. `PRIMARY`, `MINMAX`, `BLOOM_FILTER` etc. (String) -- comment - Additional information about the index, currently always `` (empty string). (String) -- index_comment - `` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String) +- comment - Additional information about the index, currently always `''` (empty string). (String) +- index_comment - `''` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String) - visible - If the index is visible to the optimizer, always `YES`. (String) -- expression - For a data skipping index, the index expression. For a primary key index: '' (empty string). (String) +- expression - For a data skipping index, the index expression. For a primary key index: `''` (empty string). (String) **Examples** @@ -313,14 +313,14 @@ SHOW INDEX FROM 'tbl' Result: ``` text -┌─table─┬─non_unique─┬─key_name─┬─column_name─┬─seq_in_index─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐ -│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │ -│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │ -│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │ -│ tbl │ 1 │ PRIMARY │ c │ 1 │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ -│ tbl │ 1 │ PRIMARY │ a │ 2 │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ -│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │ -└───────┴────────────┴──────────┴─────────────┴──────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘ +┌─table─┬─non_unique─┬─key_name─┬─seq_in_index─┬─column_name─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐ +│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │ +│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │ +│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │ +│ tbl │ 1 │ PRIMARY │ 1 │ c │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ +│ tbl │ 1 │ PRIMARY │ 2 │ a │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ +│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │ +└───────┴────────────┴──────────┴──────────────┴─────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘ ``` **See also** diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 35f32a79310..149420006fb 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -42,8 +42,8 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - arrayJoin(splitByString(', ', primary_key)) AS column_name, row_number() over (order by column_name) AS seq_in_index, + arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, NULL AS sub_part, @@ -63,8 +63,8 @@ FROM ( table AS table, 1 AS non_unique, name AS key_name, - '' AS column_name, 1 AS seq_in_index, + '' AS column_name, NULL AS collation, 0 AS cardinality, NULL AS sub_part, diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index cee0598d625..e41f2521f5c 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -1,51 +1,51 @@ --- Aliases of SHOW INDEX -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- EXTENDED -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- WHERE --- Check with weird table names -$4@^7 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -NULL 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -\' 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -\' 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES +$4@^7 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +\' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +\' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES --- Original table -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Equally named table in other database -tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES +tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES --- Short form -tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES +tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES From 8b6376005a730b9ae461d3fe93a55e51cd494181 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 20 Apr 2023 13:26:02 +0000 Subject: [PATCH 113/473] "reconfig" support for CH Keeper --- base/base/find_symbols.h | 4 +- base/base/move_extend.h | 9 + contrib/NuRaft | 2 +- programs/keeper/CMakeLists.txt | 2 + src/Common/ProfileEvents.cpp | 2 + src/Common/ZooKeeper/IKeeper.cpp | 1 + src/Common/ZooKeeper/IKeeper.h | 41 +++- src/Common/ZooKeeper/TestKeeper.cpp | 56 ++++- src/Common/ZooKeeper/TestKeeper.h | 7 + src/Common/ZooKeeper/ZooKeeper.cpp | 32 ++- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 44 +++- src/Common/ZooKeeper/ZooKeeperCommon.h | 29 +++ src/Common/ZooKeeper/ZooKeeperConstants.cpp | 50 +--- src/Common/ZooKeeper/ZooKeeperConstants.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 28 ++- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 + src/Coordination/KeeperConstants.h | 9 +- src/Coordination/KeeperContext.h | 10 +- src/Coordination/KeeperDispatcher.cpp | 103 +++++--- src/Coordination/KeeperDispatcher.h | 16 +- src/Coordination/KeeperReconfiguration.cpp | 92 +++++++ src/Coordination/KeeperReconfiguration.h | 10 + src/Coordination/KeeperServer.cpp | 232 ++++++++---------- src/Coordination/KeeperServer.h | 23 +- src/Coordination/KeeperStateMachine.cpp | 124 ++++++++-- src/Coordination/KeeperStateMachine.h | 16 +- src/Coordination/KeeperStateManager.cpp | 15 +- src/Coordination/KeeperStateManager.h | 32 +-- src/Coordination/KeeperStorage.cpp | 20 +- src/Coordination/RaftServerConfig.cpp | 96 ++++++++ src/Coordination/RaftServerConfig.h | 78 ++++++ src/Coordination/tests/gtest_coordination.cpp | 51 +++- src/Interpreters/ZooKeeperLog.cpp | 1 + src/Storages/DataLakes/HudiMetadataParser.cpp | 3 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 8 +- src/Storages/StorageReplicatedMergeTree.cpp | 16 +- tests/integration/helpers/keeper_utils.py | 33 ++- .../test_keeper_nodes_move/test.py | 5 - .../test_keeper_reconfig_add/__init__.py | 0 .../configs/keeper1.xml | 20 ++ .../configs/keeper2.xml | 21 ++ .../configs/keeper3.xml | 22 ++ .../test_keeper_reconfig_add/test.py | 155 ++++++++++++ .../test_keeper_reconfig_remove/__init__.py | 0 .../configs/keeper1.xml | 37 +++ .../configs/keeper2.xml | 37 +++ .../configs/keeper3.xml | 37 +++ .../test_keeper_reconfig_remove/test.py | 145 +++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 47 ++++ .../configs/keeper2.xml | 47 ++++ .../configs/keeper3.xml | 47 ++++ .../configs/keeper4.xml | 47 ++++ .../configs/keeper5.xml | 47 ++++ .../test_keeper_reconfig_remove_many/test.py | 149 +++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 35 +++ .../configs/keeper2.xml | 35 +++ .../configs/keeper3.xml | 35 +++ .../configs/keeper4.xml | 21 ++ .../test.py | 127 ++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 35 +++ .../configs/keeper2.xml | 35 +++ .../configs/keeper3.xml | 35 +++ .../configs/keeper4.xml | 21 ++ .../test.py | 120 +++++++++ utils/keeper-data-dumper/main.cpp | 2 +- 70 files changed, 2309 insertions(+), 367 deletions(-) create mode 100644 base/base/move_extend.h create mode 100644 src/Coordination/KeeperReconfiguration.cpp create mode 100644 src/Coordination/KeeperReconfiguration.h create mode 100644 src/Coordination/RaftServerConfig.cpp create mode 100644 src/Coordination/RaftServerConfig.h create mode 100644 tests/integration/test_keeper_reconfig_add/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_add/test.py create mode 100644 tests/integration/test_keeper_reconfig_remove/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/test.py create mode 100644 tests/integration/test_keeper_reconfig_remove_many/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/test.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/test.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index 83232669c04..f7d24ccfc11 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end) /// See https://github.com/boostorg/algorithm/issues/63 /// And https://bugs.llvm.org/show_bug.cgi?id=41141 template -inline void splitInto(To & to, const std::string & what, bool token_compress = false) +inline To& splitInto(To & to, std::string_view what, bool token_compress = false) { const char * pos = what.data(); const char * end = pos + what.size(); @@ -464,4 +464,6 @@ inline void splitInto(To & to, const std::string & what, bool token_compress = f else pos = delimiter_or_end; } + + return to; } diff --git a/base/base/move_extend.h b/base/base/move_extend.h new file mode 100644 index 00000000000..6e5b16e037c --- /dev/null +++ b/base/base/move_extend.h @@ -0,0 +1,9 @@ +#pragma once + +/// Extend @p to by moving elements from @p from to @p to end +/// @return @p to iterator to first of moved elements. +template +typename To::iterator moveExtend(To & to, From && from) +{ + return to.insert(to.end(), std::make_move_iterator(from.begin()), std::make_move_iterator(from.end())); +} diff --git a/contrib/NuRaft b/contrib/NuRaft index 491eaf592d9..eb1572129c7 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 491eaf592d950e0e37accbe8b3f217e068c9fecf +Subproject commit eb1572129c71beb2156dcdaadc3fb136954aed96 diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 18bdc8f317c..20cab03dec2 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -34,6 +34,8 @@ add_dependencies(clickhouse-keeper-lib clickhouse_keeper_configs) if (BUILD_STANDALONE_KEEPER) # Straight list of all required sources set(CLICKHOUSE_KEEPER_STANDALONE_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperReconfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/RaftServerConfig.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ACLMap.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Changelog.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/CoordinationSettings.cpp diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0838e0366df..8e3ec4f9e65 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -125,6 +125,7 @@ M(ZooKeeperMulti, "Number of 'multi' requests to ZooKeeper (compound transactions).") \ M(ZooKeeperCheck, "Number of 'check' requests to ZooKeeper. Usually they don't make sense in isolation, only as part of a complex transaction.") \ M(ZooKeeperSync, "Number of 'sync' requests to ZooKeeper. These requests are rarely needed or usable.") \ + M(ZooKeeperReconfig, "Number of 'reconfig' requests to ZooKeeper.") \ M(ZooKeeperClose, "Number of times connection with ZooKeeper has been closed voluntary.") \ M(ZooKeeperWatchResponse, "Number of times watch notification has been received from ZooKeeper.") \ M(ZooKeeperUserExceptions, "Number of exceptions while working with ZooKeeper related to the data (no node, bad version or similar).") \ @@ -499,6 +500,7 @@ The server successfully detected this situation and will download merged part fr M(KeeperCreateRequest, "Number of create requests")\ M(KeeperRemoveRequest, "Number of remove requests")\ M(KeeperSetRequest, "Number of set requests")\ + M(KeeperReconfigRequest, "Number of reconfig requests")\ M(KeeperCheckRequest, "Number of check requests")\ M(KeeperMultiRequest, "Number of multi requests")\ M(KeeperMultiReadRequest, "Number of multi read requests")\ diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index f0a07241735..50160279506 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -110,6 +110,7 @@ const char * errorMessage(Error code) case Error::ZCLOSING: return "ZooKeeper is closing"; case Error::ZNOTHING: return "(not error) no server responses to process"; case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; + case Error::ZRECONFIGINPROGRESS: return "Another reconfiguration is progress"; } UNREACHABLE(); diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 2703c1079c0..20ce2a748e6 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -82,6 +82,7 @@ enum class Error : int32_t ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments ZINVALIDSTATE = -9, /// Invalid zhandle state + ZRECONFIGINPROGRESS = -14, /// Another reconfig is running /** API errors. * This is never thrown by the server, it shouldn't be used other than @@ -350,6 +351,29 @@ struct SyncResponse : virtual Response size_t bytesSize() const override { return path.size(); } }; +struct ReconfigRequest : virtual Request +{ + String joining; + String leaving; + String new_members; + int32_t version; + + String getPath() const final { return keeper_config_path; } + + size_t bytesSize() const final + { + return joining.size() + leaving.size() + new_members.size() + sizeof(version); + } +}; + +struct ReconfigResponse : virtual Response +{ + String value; + Stat stat; + + size_t bytesSize() const override { return value.size() + sizeof(stat); } +}; + struct MultiRequest : virtual Request { Requests requests; @@ -395,9 +419,9 @@ using SetCallback = std::function; using ListCallback = std::function; using CheckCallback = std::function; using SyncCallback = std::function; +using ReconfigCallback = std::function; using MultiCallback = std::function; - /// For watches. enum State { @@ -526,6 +550,13 @@ public: const String & path, SyncCallback callback) = 0; + virtual void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) = 0; + virtual void multi( const Requests & requests, MultiCallback callback) = 0; @@ -539,3 +570,11 @@ public: }; } + +template <> struct fmt::formatter : fmt::formatter +{ + constexpr auto format(Coordination::Error code, auto& ctx) + { + return formatter::format(Coordination::errorMessage(code), ctx); + } +}; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index fe4cb83c78a..87c87c4fc92 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -3,12 +3,8 @@ #include #include #include - -#include -#include #include - namespace Coordination { @@ -147,6 +143,14 @@ struct TestKeeperSyncRequest final : SyncRequest, TestKeeperRequest std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; +struct TestKeeperReconfigRequest final : ReconfigRequest, TestKeeperRequest +{ + TestKeeperReconfigRequest() = default; + explicit TestKeeperReconfigRequest(const ReconfigRequest & base) : ReconfigRequest(base) {} + ResponsePtr createResponse() const override; + std::pair process(TestKeeper::Container & container, int64_t zxid) const override; +}; + struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest { explicit TestKeeperMultiRequest(const Requests & generic_requests) @@ -226,15 +230,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai std::string path_created = path; if (is_sequential) - { - auto seq_num = it->second.seq_num; - - std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - seq_num_str.exceptions(std::ios::failbit); - seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - - path_created += seq_num_str.str(); - } + path_created += fmt::format("{:0>10}", it->second.seq_num); /// Increment sequential number even if node is not sequential ++it->second.seq_num; @@ -446,6 +442,17 @@ std::pair TestKeeperSyncRequest::process(TestKeeper::Containe return { std::make_shared(std::move(response)), {} }; } +std::pair TestKeeperReconfigRequest::process(TestKeeper::Container &, int64_t) const +{ + // In TestKeeper we assume data is stored on one server, so this is a dummy implementation to + // satisfy IKeeper interface. + // We can't even check the validity of input data, neither can we create the /keeper/config znode + // as we don't know the id of current "server". + ReconfigResponse response; + response.error = Error::ZOK; + return { std::make_shared(std::move(response)), {} }; +} + std::pair TestKeeperMultiRequest::process(TestKeeper::Container & container, int64_t zxid) const { MultiResponse response; @@ -505,6 +512,7 @@ ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shar ResponsePtr TestKeeperListRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperCheckRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperSyncRequest::createResponse() const { return std::make_shared(); } +ResponsePtr TestKeeperReconfigRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperMultiRequest::createResponse() const { return std::make_shared(); } @@ -828,6 +836,28 @@ void TestKeeper::sync( pushRequest(std::move(request_info)); } +void TestKeeper::reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) +{ + TestKeeperReconfigRequest req; + req.joining = joining; + req.leaving = leaving; + req.new_members = new_members; + req.version = version; + + pushRequest({ + .request = std::make_shared(std::move(req)), + .callback = [callback](const Response & response) + { + callback(dynamic_cast(response)); + } + }); +} + void TestKeeper::multi( const Requests & requests, MultiCallback callback) diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 9bbd018cfb1..8615ed0fb77 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -87,6 +87,13 @@ public: const String & path, SyncCallback callback) override; + void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) final; + void multi( const Requests & requests, MultiCallback callback) override; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5dd7948276d..12b1d82133e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -75,13 +75,14 @@ void ZooKeeper::init(ZooKeeperArgs args_) auto & host_string = host.host; try { - bool secure = startsWith(host_string, "secure://"); + const bool secure = startsWith(host_string, "secure://"); if (secure) host_string.erase(0, strlen("secure://")); - LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure}); + const Poco::Net::SocketAddress host_socket_addr{host_string}; + LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, host_socket_addr.toString()); + nodes.emplace_back(Coordination::ZooKeeper::Node{host_socket_addr, secure}); } catch (const Poco::Net::HostNotFoundException & e) { @@ -191,12 +192,7 @@ std::vector ZooKeeper::shuffleHosts() const shuffle_hosts.emplace_back(shuffle_host); } - ::sort( - shuffle_hosts.begin(), shuffle_hosts.end(), - [](const ShuffleHost & lhs, const ShuffleHost & rhs) - { - return ShuffleHost::compare(lhs, rhs); - }); + ::sort(shuffle_hosts.begin(), shuffle_hosts.end(), ShuffleHost::compare); return shuffle_hosts; } @@ -231,7 +227,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::List), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::List, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -298,7 +294,7 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Create, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -393,7 +389,7 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Remove), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Remove, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -425,7 +421,7 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Exists), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Exists, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -459,7 +455,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Get), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Get, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -531,7 +527,7 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Set), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Set, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -583,7 +579,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Multi), requests[0]->getPath())); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Multi, requests[0]->getPath())); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -617,7 +613,7 @@ Coordination::Error ZooKeeper::syncImpl(const std::string & path, std::string & if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Sync), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Sync, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -1229,7 +1225,7 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: if (!Coordination::isUserError(exception_code)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There are no failed OPs because '{}' is not valid response code for that", - std::string(Coordination::errorMessage(exception_code))); + exception_code); throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There is no failed OpResult"); } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 5031af38812..c24eecbafd8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -36,7 +36,7 @@ std::string ZooKeeperRequest::toString() const "OpNum = {}\n" "Additional info:\n{}", xid, - Coordination::toString(getOpNum()), + getOpNum(), toStringImpl()); } @@ -76,6 +76,41 @@ void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(joining, out); + Coordination::write(leaving, out); + Coordination::write(new_members, out); + Coordination::write(version, out); +} + +void ZooKeeperReconfigRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(joining, in); + Coordination::read(leaving, in); + Coordination::read(new_members, in); + Coordination::read(version, in); +} + +std::string ZooKeeperReconfigRequest::toStringImpl() const +{ + return fmt::format( + "joining = {}\nleaving = {}\nnew_members = {}\nversion = {}", + joining, leaving, new_members, version); +} + +void ZooKeeperReconfigResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(value, in); + Coordination::read(stat, in); +} + +void ZooKeeperReconfigResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(value, out); + Coordination::write(stat, out); +} + void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -664,6 +699,7 @@ ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTi ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared()); } @@ -861,7 +897,8 @@ void ZooKeeperMultiResponse::fillLogElements(LogElements & elems, size_t idx) co void ZooKeeperRequestFactory::registerRequest(OpNum op_num, Creator creator) { if (!op_num_to_request.try_emplace(op_num, creator).second) - throw Coordination::Exception("Request type " + toString(op_num) + " already registered", Coordination::Error::ZRUNTIMEINCONSISTENCY); + throw Coordination::Exception(Coordination::Error::ZRUNTIMEINCONSISTENCY, + "Request type {} already registered", op_num); } std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) @@ -916,7 +953,7 @@ ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); if (it == op_num_to_request.end()) - throw Exception("Unknown operation type " + toString(op_num), Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "Unknown operation type {}", op_num); return it->second(); } @@ -960,6 +997,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 5f00698423e..131d19f1ca4 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -117,6 +117,35 @@ struct ZooKeeperSyncResponse final : SyncResponse, ZooKeeperResponse OpNum getOpNum() const override { return OpNum::Sync; } }; +struct ZooKeeperReconfigRequest final : ZooKeeperRequest +{ + String joining; + String leaving; + String new_members; + int64_t version; // kazoo sends a 64bit integer in this request + + String getPath() const override { return keeper_config_path; } + OpNum getOpNum() const override { return OpNum::Reconfig; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + std::string toStringImpl() const override; + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } + + size_t bytesSize() const override + { + return ZooKeeperRequest::bytesSize() + joining.size() + leaving.size() + new_members.size() + + sizeof(version); + } +}; + +struct ZooKeeperReconfigResponse final : ReconfigResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return OpNum::Reconfig; } +}; + struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index 86f70ea547a..9bb9c7b0488 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -19,6 +19,7 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::Heartbeat), static_cast(OpNum::List), static_cast(OpNum::Check), + static_cast(OpNum::Reconfig), static_cast(OpNum::Multi), static_cast(OpNum::MultiRead), static_cast(OpNum::Auth), @@ -29,55 +30,6 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::CheckNotExists), }; -std::string toString(OpNum op_num) -{ - switch (op_num) - { - case OpNum::Close: - return "Close"; - case OpNum::Error: - return "Error"; - case OpNum::Create: - return "Create"; - case OpNum::Remove: - return "Remove"; - case OpNum::Exists: - return "Exists"; - case OpNum::Get: - return "Get"; - case OpNum::Set: - return "Set"; - case OpNum::SimpleList: - return "SimpleList"; - case OpNum::List: - return "List"; - case OpNum::Check: - return "Check"; - case OpNum::Multi: - return "Multi"; - case OpNum::MultiRead: - return "MultiRead"; - case OpNum::Sync: - return "Sync"; - case OpNum::Heartbeat: - return "Heartbeat"; - case OpNum::Auth: - return "Auth"; - case OpNum::SessionID: - return "SessionID"; - case OpNum::SetACL: - return "SetACL"; - case OpNum::GetACL: - return "GetACL"; - case OpNum::FilteredList: - return "FilteredList"; - case OpNum::CheckNotExists: - return "CheckNotExists"; - } - int32_t raw_op = static_cast(op_num); - throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED); -} - OpNum getOpNum(int32_t raw_op_num) { if (!VALID_OPERATIONS.contains(raw_op_num)) diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 6b50c5c5d09..a773fbbab74 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -31,6 +31,7 @@ enum class OpNum : int32_t List = 12, Check = 13, Multi = 14, + Reconfig = 16, MultiRead = 22, Auth = 100, @@ -41,7 +42,6 @@ enum class OpNum : int32_t SessionID = 997, /// Special internal request }; -std::string toString(OpNum op_num); OpNum getOpNum(int32_t raw_op_num); static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION = 0; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 0f27d078234..5e16a437be3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -35,6 +35,7 @@ namespace ProfileEvents extern const Event ZooKeeperRemove; extern const Event ZooKeeperExists; extern const Event ZooKeeperMulti; + extern const Event ZooKeeperReconfig; extern const Event ZooKeeperGet; extern const Event ZooKeeperSet; extern const Event ZooKeeperList; @@ -571,7 +572,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) if (err != Error::ZOK) throw Exception(Error::ZMARSHALLINGERROR, "Error received in reply to auth request. Code: {}. Message: {}", - static_cast(err), errorMessage(err)); + static_cast(err), err); } void ZooKeeper::sendThread() @@ -697,7 +698,7 @@ void ZooKeeper::receiveThread() if (earliest_operation) { throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response in {} ms) for request {} for path: {}", - args.operation_timeout_ms, toString(earliest_operation->request->getOpNum()), earliest_operation->request->getPath()); + args.operation_timeout_ms, earliest_operation->request->getOpNum(), earliest_operation->request->getPath()); } waited_us += max_wait_us; if (waited_us >= args.session_timeout_ms * 1000) @@ -738,7 +739,7 @@ void ZooKeeper::receiveEvent() if (xid == PING_XID) { if (err != Error::ZOK) - throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", errorMessage(err)); + throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", err); response = std::make_shared(); } @@ -1195,7 +1196,6 @@ void ZooKeeper::create( ProfileEvents::increment(ProfileEvents::ZooKeeperCreate); } - void ZooKeeper::remove( const String & path, int32_t version, @@ -1335,6 +1335,26 @@ void ZooKeeper::sync( ProfileEvents::increment(ProfileEvents::ZooKeeperSync); } +void ZooKeeper::reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) +{ + ZooKeeperReconfigRequest request; + request.joining = joining; + request.leaving = leaving; + request.new_members = new_members; + request.version = version; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperReconfig); +} void ZooKeeper::multi( const Requests & requests, diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 44ea993947e..7e27608d0a1 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -178,6 +178,13 @@ public: const String & path, SyncCallback callback) override; + void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) final; + void multi( const Requests & requests, MultiCallback callback) override; diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 84cbb0ab7c5..675001d51e0 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -1,5 +1,4 @@ #pragma once - #include namespace DB @@ -14,8 +13,8 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -const std::string keeper_system_path = "/keeper"; -const std::string keeper_api_version_path = keeper_system_path + "/api_version"; -const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; - +const String keeper_system_path = "/keeper"; +const String keeper_api_version_path = keeper_system_path + "/api_version"; +const String keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; +const String keeper_config_path = keeper_system_path + "/config"; } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 229dbd51ab2..4fb552f20a3 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,8 @@ #pragma once - -#include - #include -#include #include +#include +#include #include #include @@ -12,6 +10,8 @@ namespace DB { +class KeeperDispatcher; + class KeeperContext { public: @@ -51,6 +51,7 @@ public: const KeeperFeatureFlags & getFeatureFlags() const; void dumpConfiguration(WriteBufferFromOwnString & buf) const; + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -85,6 +86,7 @@ private: std::unordered_map system_nodes_with_data; KeeperFeatureFlags feature_flags; + KeeperDispatcher * dispatcher{nullptr}; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9d9df5c7f30..178453b2f5b 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -38,6 +38,8 @@ namespace ProfileEvents extern const Event MemoryAllocatorPurgeTimeMicroseconds; } +using namespace std::chrono_literals; + namespace DB { @@ -336,6 +338,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf keeper_context = std::make_shared(standalone_keeper); keeper_context->initialize(config); + keeper_context->dispatcher = this; server = std::make_unique( configuration_and_settings, @@ -392,7 +395,10 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf /// Start it after keeper server start session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); - update_configuration_thread = ThreadFromGlobalPool([this] { updateConfigurationThread(); }); + + update_configuration_thread = reconfigEnabled() + ? ThreadFromGlobalPool([this] { clusterUpdateThread(); }) + : ThreadFromGlobalPool([this] { clusterUpdateWithReconfigDisabledThread(); }); LOG_DEBUG(log, "Dispatcher initialized"); } @@ -429,7 +435,7 @@ void KeeperDispatcher::shutdown() if (snapshot_thread.joinable()) snapshot_thread.join(); - update_configuration_queue.finish(); + cluster_update_queue.finish(); if (update_configuration_thread.joinable()) update_configuration_thread.join(); } @@ -608,7 +614,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession "Could not push error response xid {} zxid {} error message {} to responses queue", response->xid, response->zxid, - errorMessage(error)); + error); } } @@ -653,7 +659,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) { if (response->getOpNum() != Coordination::OpNum::SessionID) promise->set_exception(std::make_exception_ptr(Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect response of type {} instead of SessionID response", Coordination::toString(response->getOpNum())))); + "Incorrect response of type {} instead of SessionID response", response->getOpNum()))); auto session_id_response = dynamic_cast(*response); if (session_id_response.internal_id != internal_id) @@ -685,17 +691,12 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) return future.get(); } - -void KeeperDispatcher::updateConfigurationThread() +void KeeperDispatcher::clusterUpdateWithReconfigDisabledThread() { - while (true) + while (!shutdown_called) { - if (shutdown_called) - return; - try { - using namespace std::chrono_literals; if (!server->checkInit()) { LOG_INFO(log, "Server still not initialized, will not apply configuration until initialization finished"); @@ -710,11 +711,10 @@ void KeeperDispatcher::updateConfigurationThread() continue; } - ConfigUpdateAction action; - if (!update_configuration_queue.pop(action)) + ClusterUpdateAction action; + if (!cluster_update_queue.pop(action)) break; - /// We must wait this update from leader or apply it ourself (if we are leader) bool done = false; while (!done) @@ -727,15 +727,13 @@ void KeeperDispatcher::updateConfigurationThread() if (isLeader()) { - server->applyConfigurationUpdate(action); + server->applyConfigUpdateWithReconfigDisabled(action); done = true; } - else - { - done = server->waitConfigurationUpdate(action); - if (!done) - LOG_INFO(log, "Cannot wait for configuration update, maybe we become leader, or maybe update is invalid, will try to wait one more time"); - } + else if (done = server->waitForConfigUpdateWithReconfigDisabled(action); !done) + LOG_INFO(log, + "Cannot wait for configuration update, maybe we became leader " + "or maybe update is invalid, will try to wait one more time"); } } catch (...) @@ -745,6 +743,46 @@ void KeeperDispatcher::updateConfigurationThread() } } +void KeeperDispatcher::clusterUpdateThread() +{ + while (!shutdown_called) + { + ClusterUpdateAction action; + if (!cluster_update_queue.pop(action)) + return; + + if (server->applyConfigUpdate(action)) + LOG_DEBUG(log, "Processing config update {}: accepted", action); + else // TODO (myrrc) sleep a random amount? sleep less? + { + (void)cluster_update_queue.pushFront(action); + LOG_DEBUG(log, "Processing config update {}: declined, backoff", action); + std::this_thread::sleep_for(50ms); + } + } +} + +void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions&& actions) +{ + if (shutdown_called) return; + for (auto && action : actions) + { + if (!cluster_update_queue.push(std::move(action))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push configuration update"); + LOG_DEBUG(log, "Processing config update {}: pushed", action); + } +} + +bool KeeperDispatcher::clusterUpdateQueueEmpty() const +{ + return cluster_update_queue.empty(); +} + +bool KeeperDispatcher::reconfigEnabled() const +{ + return server->reconfigEnabled(); +} + bool KeeperDispatcher::isServerActive() const { return checkInit() && hasLeader() && !server->isRecovering(); @@ -752,20 +790,25 @@ bool KeeperDispatcher::isServerActive() const void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) { - auto diff = server->getConfigurationDiff(config); + auto diff = server->getRaftConfigurationDiff(config); + if (diff.empty()) - LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT"); + LOG_TRACE(log, "Configuration update triggered, but nothing changed for Raft"); + else if (reconfigEnabled()) + LOG_WARNING(log, + "Raft configuration changed, but keeper_server.enable_reconfiguration is on. " + "This update will be ignored. Use \"reconfig\" instead"); else if (diff.size() > 1) - LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); + LOG_WARNING(log, + "Configuration changed for more than one server ({}) from cluster, " + "it's strictly not recommended", diff.size()); else LOG_DEBUG(log, "Configuration change size ({})", diff.size()); - for (auto & change : diff) - { - bool push_result = update_configuration_queue.push(change); - if (!push_result) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); - } + if (!reconfigEnabled()) + for (auto & change : diff) + if (!cluster_update_queue.push(change)) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); snapshot_s3.updateS3Configuration(config, macros); } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 1b44f0f6ced..a9b3d33eb51 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -31,7 +31,7 @@ private: using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; - using UpdateConfigurationQueue = ConcurrentBoundedQueue; + using ClusterUpdateQueue = ConcurrentBoundedQueue; /// Size depends on coordination settings std::unique_ptr requests_queue; @@ -39,7 +39,7 @@ private: SnapshotsQueue snapshots_queue{1}; /// More than 1k updates is definitely misconfiguration. - UpdateConfigurationQueue update_configuration_queue{1000}; + ClusterUpdateQueue cluster_update_queue{1000}; std::atomic shutdown_called{false}; @@ -91,8 +91,10 @@ private: void sessionCleanerTask(); /// Thread create snapshots in the background void snapshotThread(); - /// Thread apply or wait configuration changes from leader - void updateConfigurationThread(); + + // TODO (myrrc) this should be removed once "reconfig" is stabilized + void clusterUpdateWithReconfigDisabledThread(); + void clusterUpdateThread(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); @@ -132,10 +134,10 @@ public: /// and achieved quorum bool isServerActive() const; - /// Registered in ConfigReloader callback. Add new configuration changes to - /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. - /// 'macros' are used to substitute macros in endpoint of disks void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); + void pushClusterUpdates(ClusterUpdateActions&& actions); + bool clusterUpdateQueueEmpty() const; + bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp new file mode 100644 index 00000000000..dec3e1f155f --- /dev/null +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -0,0 +1,92 @@ +#include "KeeperReconfiguration.h" +#include +#include +#include +#include + +namespace DB +{ +ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining) +{ + ClusterUpdateActions out; + std::unordered_set endpoints; + + for (const auto & server : cfg->get_servers()) + endpoints.emplace(server->get_endpoint()); + + // We can either add new servers or change weight of existing ones. + // It makes no sense having a server in _joining_ which is identical to existing one including + // weight, so such requests are declined. + for (const RaftServerConfig & update : parseRaftServers(joining)) + if (auto server_ptr = cfg->get_server(update.id)) + { + if (update.endpoint != server_ptr->get_endpoint() || update.learner != server_ptr->is_learner() + || update.priority == server_ptr->get_priority()) + return {}; // can't change server endpoint/type due to NuRaft API limitations + out.emplace_back(UpdateRaftServerPriority{.id = update.id, .priority = update.priority}); + } + else if (endpoints.contains(update.endpoint)) + return {}; + else + out.emplace_back(AddRaftServer{update}); + + return out; +} + +ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving) +{ + std::vector leaving_arr; + splitInto<','>(leaving_arr, leaving); + if (leaving_arr.size() >= cfg->get_servers().size()) + return {}; + + std::unordered_set remove_ids; + ClusterUpdateActions out; + + for (std::string_view leaving_server : leaving_arr) + { + int id; + if (std::from_chars(leaving_server.begin(), leaving_server.end(), id).ec != std::error_code{}) + return {}; + + if (remove_ids.contains(id)) + continue; + + if (auto ptr = cfg->get_server(id)) + out.emplace_back(RemoveRaftServer{.id = id}); + else + return {}; + + remove_ids.emplace(id); + } + + return out; +} + +String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates) +{ + RaftServers new_config; + std::unordered_set remove_update_ids; + + for (const auto & update : updates) + { + if (const auto * add = std::get_if(&update)) + new_config.emplace_back(*add); + else if (const auto * remove = std::get_if(&update)) + remove_update_ids.insert(remove->id); + else if (const auto * priority = std::get_if(&update)) + { + remove_update_ids.insert(priority->id); + new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)}); + } + else + UNREACHABLE(); + } + + for (const auto & item : cfg->get_servers()) + if (!remove_update_ids.contains(item->get_id())) + new_config.emplace_back(RaftServerConfig{*item}); + + return fmt::format("{}", fmt::join(new_config.begin(), new_config.end(), "\n")); +} +} diff --git a/src/Coordination/KeeperReconfiguration.h b/src/Coordination/KeeperReconfiguration.h new file mode 100644 index 00000000000..71958f2035e --- /dev/null +++ b/src/Coordination/KeeperReconfiguration.h @@ -0,0 +1,10 @@ +#pragma once +#include "Coordination/KeeperSnapshotManager.h" +#include "Coordination/RaftServerConfig.h" + +namespace DB +{ +ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining); +ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving); +String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates = {}); +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 82c843287c1..1cde957ef3a 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB { @@ -40,6 +41,8 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } +using namespace std::chrono_literals; + namespace { @@ -118,6 +121,7 @@ KeeperServer::KeeperServer( , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) + , enable_reconfiguration(config.getBool("keeper_server.enable_reconfiguration", false)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); @@ -450,7 +454,7 @@ void KeeperServer::shutdownRaftServer() size_t count = 0; while (asio_service->get_active_workers() != 0 && count < timeout * 100) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); + std::this_thread::sleep_for(10ms); count++; } } @@ -715,10 +719,12 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (next_index < last_commited || next_index - last_commited <= 1) commited_store = true; - auto set_initialized = [this]() + auto set_initialized = [this] { - std::lock_guard lock(initialized_mutex); - initialized_flag = true; + { + std::lock_guard lock(initialized_mutex); + initialized_flag = true; + } initialized_cv.notify_all(); }; @@ -783,9 +789,42 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) +bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) { - auto diff = state_manager->getConfigurationDiff(config); + std::lock_guard _{server_write_mutex}; + + if (const auto* add = std::get_if(&action)) + return raft_instance->get_srv_config(add->id) != nullptr + || raft_instance->add_srv(static_cast(*add))->get_accepted(); + else if (const auto* remove = std::get_if(&action)) + { + if (isLeader() && remove->id == state_manager->server_id()) + { + raft_instance->yield_leadership(); + return false; + } + + return raft_instance->get_srv_config(remove->id) == nullptr + || raft_instance->remove_srv(remove->id)->get_accepted(); + } + else if (const auto* update = std::get_if(&action)) + { + if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr) + throw Exception(ErrorCodes::RAFT_ERROR, + "Attempt to apply {} but server is not present in Raft", + action); + else if (ptr->get_priority() == update->priority) + return true; + + raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); + return true; + } + UNREACHABLE(); +} + +ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) +{ + auto diff = state_manager->getRaftConfigurationDiff(config); if (!diff.empty()) { @@ -796,160 +835,103 @@ ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::Abstrac return diff; } -void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task) +void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action) { - std::lock_guard lock{server_write_mutex}; - if (is_recovering) - return; + std::lock_guard _{server_write_mutex}; + if (is_recovering) return; + constexpr auto sleep_time = 500ms; - size_t sleep_ms = 500; - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to apply {}", action); + + auto applied = [&] { LOG_INFO(log, "Applied {}", action); }; + auto not_leader = [&] { LOG_INFO(log, "Not leader anymore, aborting"); }; + auto backoff_on_refusal = [&](size_t i) + { + LOG_INFO(log, "Update was not accepted (try {}), backing off for {}", i + 1, sleep_time * (i + 1)); + std::this_thread::sleep_for(sleep_time * (i + 1)); + }; + + if (const auto* add = std::get_if(&action)) { - LOG_INFO(log, "Will try to add server with id {}", task.server->get_id()); - bool added = false; for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) - { - LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id()); - added = true; - break; - } - + if (raft_instance->get_srv_config(add->id) != nullptr) + return applied(); if (!isLeader()) - { - LOG_INFO(log, "We are not leader anymore, will not try to add server {}", task.server->get_id()); - break; - } - - auto result = raft_instance->add_srv(*task.server); - if (!result->get_accepted()) - LOG_INFO( - log, - "Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry", - task.server->get_id(), - i + 1, - sleep_ms * (i + 1)); - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return not_leader(); + if (!raft_instance->add_srv(static_cast(*add))->get_accepted()) + backoff_on_refusal(i); } - if (!added) - throw Exception( - ErrorCodes::RAFT_ERROR, - "Configuration change to add server (id {}) was not accepted by RAFT after all {} retries", - task.server->get_id(), - coordination_settings->configuration_change_tries_count); } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + else if (const auto* remove = std::get_if(&action)) { - LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); - - bool removed = false; - if (task.server->get_id() == state_manager->server_id()) + if (remove->id == state_manager->server_id()) { - LOG_INFO( - log, - "Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. " + LOG_INFO(log, + "Trying to remove leader node (ourself), so will yield leadership and some other node " + "(new leader) will try to remove us. " "Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node"); - - raft_instance->yield_leadership(); - return; + return raft_instance->yield_leadership(); } for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) - { - LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id()); - removed = true; - break; - } - + if (raft_instance->get_srv_config(remove->id) == nullptr) + return applied(); if (!isLeader()) - { - LOG_INFO(log, "We are not leader anymore, will not try to remove server {}", task.server->get_id()); - break; - } - - auto result = raft_instance->remove_srv(task.server->get_id()); - if (!result->get_accepted()) - LOG_INFO( - log, - "Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry", - task.server->get_id(), - i + 1, - sleep_ms * (i + 1)); - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return not_leader(); + if (!raft_instance->remove_srv(remove->id)->get_accepted()) + backoff_on_refusal(i); } - if (!removed) - throw Exception( - ErrorCodes::RAFT_ERROR, - "Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries", - task.server->get_id(), - coordination_settings->configuration_change_tries_count); } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) - raft_instance->set_priority(task.server->get_id(), task.server->get_priority()); - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); + else if (const auto* update = std::get_if(&action)) + { + raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); + return; + } + + throw Exception(ErrorCodes::RAFT_ERROR, + "Configuration change {} was not accepted by Raft after {} retries", + action, coordination_settings->configuration_change_tries_count); } - -bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task) +bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action) { - if (is_recovering) - return false; + if (is_recovering) return false; + constexpr auto sleep_time = 500ms; - size_t sleep_ms = 500; - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to wait for {}", action); + + auto applied = [&] { LOG_INFO(log, "Applied {}", action); return true; }; + auto became_leader = [&] { LOG_INFO(log, "Became leader, aborting"); return false; }; + auto backoff = [&](size_t i) { std::this_thread::sleep_for(sleep_time * (i + 1)); }; + + if (const auto* add = std::get_if(&action)) { - LOG_INFO(log, "Will try to wait server with id {} to be added", task.server->get_id()); for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) - { - LOG_INFO(log, "Server with id {} was successfully added by leader", task.server->get_id()); - return true; - } - + if (raft_instance->get_srv_config(add->id) != nullptr) + return applied(); if (isLeader()) - { - LOG_INFO(log, "We are leader now, probably we will have to add server {}", task.server->get_id()); - return false; - } - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return became_leader(); + backoff(i); } - return false; } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + else if (const auto* remove = std::get_if(&action)) { - LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id()); - for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) - { - LOG_INFO(log, "Server with id {} was successfully removed by leader", task.server->get_id()); - return true; - } - + if (raft_instance->get_srv_config(remove->id) == nullptr) + return applied(); if (isLeader()) - { - LOG_INFO(log, "We are leader now, probably we will have to remove server {}", task.server->get_id()); - return false; - } - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return became_leader(); + backoff(i); } - return false; } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) + else if (std::get_if(&action) != nullptr) return true; - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); - return true; + + return false; } Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 8f416b1f48c..61e29b67bbd 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -10,12 +10,15 @@ #include #include #include +#include namespace DB { using RaftAppendResult = nuraft::ptr>>; +class KeeperDispatcher; + class KeeperServer { private: @@ -28,9 +31,10 @@ private: nuraft::ptr state_manager; struct KeeperRaftServer; - nuraft::ptr raft_instance; + nuraft::ptr raft_instance; // TSA_GUARDED_BY(server_write_mutex); nuraft::ptr asio_service; std::vector> asio_listeners; + // because some actions can be applied // when we are sure that there are no requests currently being // processed (e.g. recovery) we do all write actions @@ -65,6 +69,7 @@ private: std::shared_ptr keeper_context; const bool create_snapshot_on_exit; + const bool enable_reconfiguration; public: KeeperServer( @@ -84,6 +89,7 @@ public: void putLocalReadRequest(const KeeperStorage::RequestForSession & request); bool isRecovering() const { return is_recovering; } + bool reconfigEnabled() const { return enable_reconfiguration; } /// Put batch of requests into Raft and get result of put. Responses will be set separately into /// responses_queue. @@ -122,17 +128,12 @@ public: int getServerID() const { return server_id; } - /// Get configuration diff between current configuration in RAFT and in XML file - ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); + bool applyConfigUpdate(const ClusterUpdateAction& action); - /// Apply action for configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. - /// Synchronously check for update results with retries. - void applyConfigurationUpdate(const ConfigUpdateAction & task); - - - /// Wait configuration update for action. Used by followers. - /// Return true if update was successfully received. - bool waitConfigurationUpdate(const ConfigUpdateAction & task); + // TODO (myrrc) these functions should be removed once "reconfig" is stabilized + void applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action); + bool waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action); + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config); uint64_t createSnapshot(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5c84f23fc60..3e9850caa40 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -2,17 +2,20 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include #include #include #include "Coordination/KeeperStorage.h" +#include "Coordination/KeeperReconfiguration.h" #include @@ -146,7 +149,7 @@ void assertDigest( "Digest for nodes is not matching after {} request of type '{}'.\nExpected digest - {}, actual digest - {} (digest " "{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}", committing ? "committing" : "preprocessing", - Coordination::toString(request.getOpNum()), + request.getOpNum(), first.value, second.value, first.version, @@ -261,7 +264,8 @@ std::shared_ptr KeeperStateMachine::parseReque bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) { - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + const auto op_num = request_for_session.request->getOpNum(); + if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; std::lock_guard lock(storage_and_responses_lock); @@ -291,14 +295,89 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } +KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( + const KeeperStorage::RequestForSession& request_for_session) +{ + const auto& request = static_cast(*request_for_session.request); + const int64_t session_id = request_for_session.session_id; + const int64_t zxid = request_for_session.zxid; + + using enum Coordination::Error; + auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession + { + auto res = std::make_shared(); + res->xid = request.xid; + res->zxid = zxid; + res->error = code; + return { session_id, std::move(res) }; + }; + + KeeperDispatcher& dispatcher = *keeper_context->dispatcher; + if (!dispatcher.reconfigEnabled()) + return bad_request(ZUNIMPLEMENTED); + if (!dispatcher.clusterUpdateQueueEmpty()) + return bad_request(ZRECONFIGINPROGRESS); + if (request.version != -1) + return bad_request(ZBADVERSION); + + const bool has_new_members = !request.new_members.empty(); + const bool has_joining = !request.joining.empty(); + const bool has_leaving = !request.leaving.empty(); + const bool incremental_reconfig = (has_joining || has_leaving) && !has_new_members; + if (!incremental_reconfig) + return bad_request(); + + const ClusterConfigPtr config = getClusterConfig(); + if (!config) // Server can be uninitialized yet + return bad_request(); + + ClusterUpdateActions updates; + + if (has_joining) + { + if (auto join_updates = joiningToClusterUpdates(config, request.joining); !join_updates.empty()) + moveExtend(updates, std::move(join_updates)); + else + return bad_request(); + } + + if (has_leaving) + { + if (auto leave_updates = leavingToClusterUpdates(config, request.leaving); !leave_updates.empty()) + moveExtend(updates, std::move(leave_updates)); + else + return bad_request(); + } + + auto response = std::make_shared(); + response->xid = request.xid; + response->zxid = zxid; + response->error = Coordination::Error::ZOK; + response->value = serializeClusterConfig(config, updates); + + dispatcher.pushClusterUpdates(std::move(updates)); + return { session_id, std::move(response) }; +} + nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, true); if (!request_for_session->zxid) request_for_session->zxid = log_idx; - /// Special processing of session_id request - if (request_for_session->request->getOpNum() == Coordination::OpNum::SessionID) + auto try_push = [this](const KeeperStorage::ResponseForSession& response) + { + if (!responses_queue.push(response)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); + LOG_WARNING(log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response.session_id); + } + }; + + const auto op_num = request_for_session->request->getOpNum(); + if (op_num == Coordination::OpNum::SessionID) { const Coordination::ZooKeeperSessionIDRequest & session_id_request = dynamic_cast(*request_for_session->request); @@ -309,21 +388,24 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponseForSession response_for_session; response_for_session.session_id = -1; response_for_session.response = response; - { - std::lock_guard lock(storage_and_responses_lock); - session_id = storage->getSessionID(session_id_request.session_timeout_ms); - LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); - response->session_id = session_id; - if (!responses_queue.push(response_for_session)) - { - ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", session_id); - } - } + + std::lock_guard lock(storage_and_responses_lock); + session_id = storage->getSessionID(session_id_request.session_timeout_ms); + LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); + response->session_id = session_id; + try_push(response_for_session); + } + // Processing reconfig request as an ordinary one (in KeeperStorage) brings multiple inconsistencies + // regarding replays of old reconfigurations in new nodes. Thus the storage is not involved. + // See https://github.com/ClickHouse/ClickHouse/pull/49450 for details + else if (op_num == Coordination::OpNum::Reconfig) + { + std::lock_guard lock(storage_and_responses_lock); + try_push(processReconfiguration(*request_for_session)); } else { - if (request_for_session->request->getOpNum() == Coordination::OpNum::Close) + if (op_num == Coordination::OpNum::Close) { std::lock_guard lock(request_cache_mutex); parsed_request_cache.erase(request_for_session->session_id); @@ -333,14 +415,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) - if (!responses_queue.push(response_for_session)) - { - ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING( - log, - "Failed to push response with session id {} to the queue, probably because of shutdown", - response_for_session.session_id); - } + try_push(response_for_session); if (keeper_context->digestEnabled() && request_for_session->digest) assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true); @@ -782,5 +857,4 @@ void KeeperStateMachine::recalculateStorageStats() storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); } - } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index b47a9b5cc42..3b239adae45 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,6 +12,7 @@ namespace DB { +class KeeperDispatcher; using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; @@ -67,7 +68,9 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); - void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + void rollbackRequestNoLock( + const KeeperStorage::RequestForSession & request_for_session, + bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS; uint64_t last_commit_index() override { return last_committed_idx; } @@ -87,8 +90,10 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - /// just for test - KeeperStorage & getStorage() { return *storage; } + KeeperStorage & getStorageForUnitTests() TSA_NO_THREAD_SAFETY_ANALYSIS + { + return *storage; + } void shutdownStorage(); @@ -122,6 +127,7 @@ public: uint64_t getLatestSnapshotBufSize() const; void recalculateStorageStats(); + private: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored @@ -133,7 +139,7 @@ private: CoordinationSettingsPtr coordination_settings; /// Main state machine logic - KeeperStoragePtr storage; + KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; @@ -178,6 +184,8 @@ private: KeeperContextPtr keeper_context; KeeperSnapshotManagerS3 * snapshot_manager_s3; + + KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession& request_for_session); }; } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 450fd04b61d..cf1bad8c5fa 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -451,7 +451,7 @@ nuraft::ptr KeeperStateManager::read_state() return nullptr; } -ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const +ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const { auto new_configuration_wrapper = parseServersConfiguration(config, true); @@ -465,14 +465,14 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A old_ids[old_server->get_id()] = old_server; } - ConfigUpdateActions result; + ClusterUpdateActions result; /// First of all add new servers for (const auto & [new_id, server_config] : new_ids) { auto old_server_it = old_ids.find(new_id); if (old_server_it == old_ids.end()) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); + result.emplace_back(AddRaftServer{RaftServerConfig{*server_config}}); else { const auto & old_endpoint = old_server_it->second->get_endpoint(); @@ -491,10 +491,8 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A /// After that remove old ones for (auto [old_id, server_config] : old_ids) - { if (!new_ids.contains(old_id)) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); - } + result.emplace_back(RemoveRaftServer{old_id}); { std::lock_guard lock(configuration_wrapper_mutex); @@ -507,7 +505,10 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A { if (old_server->get_priority() != new_server->get_priority()) { - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + result.emplace_back(UpdateRaftServerPriority{ + .id = new_server->get_id(), + .priority = new_server->get_priority() + }); } break; } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index f24f0c2b1e5..5abeea604b5 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -7,31 +7,13 @@ #include #include #include "Coordination/KeeperStateMachine.h" +#include "Coordination/RaftServerConfig.h" #include namespace DB { - using KeeperServerConfigPtr = nuraft::ptr; -/// When our configuration changes the following action types -/// can happen -enum class ConfigUpdateActionType -{ - RemoveServer, - AddServer, - UpdatePriority, -}; - -/// Action to update configuration -struct ConfigUpdateAction -{ - ConfigUpdateActionType action_type; - KeeperServerConfigPtr server; -}; - -using ConfigUpdateActions = std::vector; - /// Responsible for managing our and cluster configuration class KeeperStateManager : public nuraft::state_mgr { @@ -74,7 +56,11 @@ public: int32_t server_id() override { return my_server_id; } - nuraft::ptr get_srv_config() const { return configuration_wrapper.config; } /// NOLINT + nuraft::ptr get_srv_config() const + { + std::lock_guard lk(configuration_wrapper_mutex); + return configuration_wrapper.config; + } void system_exit(const int exit_code) override; /// NOLINT @@ -106,8 +92,8 @@ public: /// Read all log entries in log store from the begging and return latest config (with largest log_index) ClusterConfigPtr getLatestConfigFromLogStore() const; - /// Get configuration diff between proposed XML and current state in RAFT - ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; + // TODO (myrrc) This should be removed once "reconfig" is stabilized + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: const String & getOldServerStatePath(); @@ -133,7 +119,7 @@ private: std::string config_prefix; mutable std::mutex configuration_wrapper_mutex; - KeeperConfigurationWrapper configuration_wrapper; + KeeperConfigurationWrapper configuration_wrapper TSA_GUARDED_BY(configuration_wrapper_mutex); nuraft::ptr log_store; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 884aacc4558..2b245a455b7 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -20,10 +20,10 @@ #include #include +#include #include +#include -#include -#include #include #include #include @@ -53,7 +53,6 @@ namespace ErrorCodes namespace { - String getSHA1(const String & userdata) { Poco::SHA1Engine engine; @@ -1060,7 +1059,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - if (request.path == Coordination::keeper_api_feature_flags_path) + if (request.path == Coordination::keeper_api_feature_flags_path + || request.path == Coordination::keeper_config_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) @@ -1085,6 +1085,14 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } } + if (request.path == Coordination::keeper_config_path) + { + response.data = serializeClusterConfig( + storage.keeper_context->dispatcher->getStateMachine().getClusterConfig()); + response.error = Coordination::Error::ZOK; + return response_ptr; + } + auto & container = storage.container; auto node_it = container.find(request.path); if (node_it == container.end()) @@ -1784,7 +1792,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", - Coordination::toString(sub_zk_request->getOpNum())); + sub_zk_request->getOpNum()); } } @@ -1975,7 +1983,7 @@ public: { auto request_it = op_num_to_request.find(zk_request->getOpNum()); if (request_it == op_num_to_request.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", toString(zk_request->getOpNum())); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", zk_request->getOpNum()); return request_it->second(zk_request); } diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp new file mode 100644 index 00000000000..42923dd0b29 --- /dev/null +++ b/src/Coordination/RaftServerConfig.cpp @@ -0,0 +1,96 @@ +#include "RaftServerConfig.h" +#include +#include +#include +#include + +namespace DB +{ +RaftServerConfig::RaftServerConfig(const nuraft::srv_config & cfg) noexcept + : id(cfg.get_id()), endpoint(cfg.get_endpoint()), learner(cfg.is_learner()), priority(cfg.get_priority()) +{ +} + +RaftServerConfig::operator nuraft::srv_config() const noexcept +{ + return {id, 0, endpoint, "", learner, priority}; +} + +std::optional RaftServerConfig::parse(std::string_view server) noexcept +{ + std::vector parts; + splitInto<';', '='>(parts, server); + + const bool with_id_endpoint = parts.size() == 2; + const bool with_server_type = parts.size() == 3; + const bool with_priority = parts.size() == 4; + if (!with_id_endpoint && !with_server_type && !with_priority) + return std::nullopt; + + const std::string_view id_str = parts[0]; + if (!id_str.starts_with("server.")) + return std::nullopt; + + int id; + if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) + return std::nullopt; + if (id <= 0) + return std::nullopt; + + const std::string_view endpoint = parts[1]; + const size_t port_delimiter = endpoint.find_last_of(':'); + if (port_delimiter == std::string::npos) + return {}; + const std::string_view port = endpoint.substr(port_delimiter + 1); + + uint16_t port_tmp; + if (std::from_chars(port.begin(), port.end(), port_tmp).ec != std::error_code{}) + return std::nullopt; + + RaftServerConfig out{id, endpoint}; + + if (with_id_endpoint) + return out; + + if (parts[2] != "learner" && parts[2] != "participant") + return std::nullopt; + out.learner = parts[2] == "learner"; + if (with_server_type) + return out; + + const std::string_view priority = parts[3]; + if (std::from_chars(priority.begin(), priority.end(), out.priority).ec != std::error_code{}) + return std::nullopt; + if (out.priority < 0) + return std::nullopt; + + return out; +} + +RaftServers parseRaftServers(std::string_view servers) +{ + std::vector server_arr; + std::unordered_set ids; + std::unordered_set endpoints; + RaftServers out; + + for (auto & server : splitInto<','>(server_arr, servers)) + if (auto maybe_server = RaftServerConfig::parse(server)) + { + String endpoint = maybe_server->endpoint; + if (endpoints.contains(endpoint)) + return {}; + const int id = maybe_server->id; + if (ids.contains(id)) + return {}; + + out.emplace_back(std::move(*maybe_server)); + endpoints.emplace(std::move(endpoint)); + ids.emplace(id); + } + else + return {}; + + return out; +} +} diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h new file mode 100644 index 00000000000..451d61a436e --- /dev/null +++ b/src/Coordination/RaftServerConfig.h @@ -0,0 +1,78 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ +// default- and copy-constructible version of nuraft::srv_config +struct RaftServerConfig +{ + int id; + String endpoint; + bool learner; + int priority; + + constexpr RaftServerConfig() = default; + constexpr RaftServerConfig(int id_, std::string_view endpoint_, bool learner_ = false, int priority_ = 1) + : id(id_), endpoint(endpoint_), learner(learner_), priority(priority_) + { + } + + constexpr bool operator==(const RaftServerConfig &) const = default; + explicit RaftServerConfig(const nuraft::srv_config & cfg) noexcept; + explicit operator nuraft::srv_config() const noexcept; + + /// Parse server in format "server.id=host:port[;learner][;priority]" + static std::optional parse(std::string_view server) noexcept; +}; + +using RaftServers = std::vector; +/// Parse comma-delimited servers. Check for duplicate endpoints and ids. +/// @returns {} on parsing or validation error. +RaftServers parseRaftServers(std::string_view servers); + +struct AddRaftServer : RaftServerConfig +{ +}; + +struct RemoveRaftServer +{ + int id; +}; + +struct UpdateRaftServerPriority +{ + int id; + int priority; +}; + +using ClusterUpdateAction = std::variant; +using ClusterUpdateActions = std::vector; +} + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) + { + return fmt::format_to( + ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority); + } +}; + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) + { + if (const auto * add = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Add server {})", add->id); + if (const auto * remove = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Remove server {})", remove->id); + if (const auto * update = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Change server {} priority to {})", update->id, update->priority); + UNREACHABLE(); + } +}; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 0f60c960b8b..b302f9b13ca 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -84,6 +84,47 @@ protected: } }; +TEST_P(CoordinationTest, RaftServerConfigParse) +{ + auto parse = Coordination::RaftServerConfig::parse; + using Cfg = std::optional; + + EXPECT_EQ(parse(""), std::nullopt); + EXPECT_EQ(parse("="), std::nullopt); + EXPECT_EQ(parse("=;"), std::nullopt); + EXPECT_EQ(parse("=;;"), std::nullopt); + EXPECT_EQ(parse("=:80"), std::nullopt); + EXPECT_EQ(parse("server."), std::nullopt); + EXPECT_EQ(parse("server.=:80"), std::nullopt); + EXPECT_EQ(parse("server.-5=1:2"), std::nullopt); + EXPECT_EQ(parse("server.1=host;-123"), std::nullopt); + EXPECT_EQ(parse("server.1=host:999"), (Cfg{{1, "host:999"}})); + EXPECT_EQ(parse("server.1=host:999;learner"), (Cfg{{1, "host:999", true}})); + EXPECT_EQ(parse("server.1=host:999;participant"), (Cfg{{1, "host:999", false}})); + EXPECT_EQ(parse("server.1=host:999;learner;25"), (Cfg{{1, "host:999", true, 25}})); + + EXPECT_EQ(parse("server.1=127.0.0.1:80"), (Cfg{{1, "127.0.0.1:80"}})); + EXPECT_EQ( + parse("server.1=2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"), + (Cfg{{1, "2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"}})); +} + +TEST_P(CoordinationTest, RaftServerClusterConfigParse) +{ + auto parse = Coordination::parseRaftServers; + using Cfg = DB::RaftServerConfig; + using Servers = DB::RaftServers; + + EXPECT_EQ(parse(""), Servers{}); + EXPECT_EQ(parse(","), Servers{}); + EXPECT_EQ(parse("1,2"), Servers{}); + EXPECT_EQ(parse("server.1=host:80,server.1=host2:80"), Servers{}); + EXPECT_EQ(parse("server.1=host:80,server.2=host:80"), Servers{}); + EXPECT_EQ( + parse("server.1=host:80,server.2=host:81"), + (Servers{Cfg{1, "host:80"}, Cfg{2, "host:81"}})); +} + TEST_P(CoordinationTest, BuildTest) { DB::InMemoryLogStore store; @@ -1575,8 +1616,8 @@ void testLogAndStateMachine( restore_machine->commit(i, changelog.entry_at(i)->get_buf()); } - auto & source_storage = state_machine->getStorage(); - auto & restored_storage = restore_machine->getStorage(); + auto & source_storage = state_machine->getStorageForUnitTests(); + auto & restored_storage = restore_machine->getStorageForUnitTests(); EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1678,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); state_machine->pre_commit(1, entry_c->get_buf()); state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorage(); + const auto & storage = state_machine->getStorageForUnitTests(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); @@ -1727,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); state_machine->pre_commit(2, create_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); // commit log entries @@ -1790,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) state_machine->commit(2, create_entry->get_buf()); state_machine->commit(3, set_acl_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; auto node = uncommitted_state.getNode(node_path); ASSERT_NE(node, nullptr); diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 48f4d510af7..2231a58c6a9 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -73,6 +73,7 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes() {"Create", static_cast(Coordination::OpNum::Create)}, {"Remove", static_cast(Coordination::OpNum::Remove)}, {"Exists", static_cast(Coordination::OpNum::Exists)}, + {"Reconfig", static_cast(Coordination::OpNum::Reconfig)}, {"Get", static_cast(Coordination::OpNum::Get)}, {"Set", static_cast(Coordination::OpNum::Set)}, {"GetACL", static_cast(Coordination::OpNum::GetACL)}, diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/DataLakes/HudiMetadataParser.cpp index a1f35a5ae42..78d69c83989 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ b/src/Storages/DataLakes/HudiMetadataParser.cpp @@ -67,7 +67,8 @@ struct HudiMetadataParser::Impl { auto key_file = std::filesystem::path(key); Strings file_parts; - splitInto<'_'>(file_parts, key_file.stem()); + const String stem = key_file.stem(); + splitInto<'_'>(file_parts, stem); if (file_parts.size() != 3) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index bcc4dc749fb..07cfced8362 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -492,7 +492,7 @@ size_t ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_di } else { - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, Coordination::errorMessage(rc)); + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, rc); } first_outdated_block++; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 792843cbe18..b08b9de12a3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -494,7 +494,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper( if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Couldn't set value of nodes for insert times " "({}/min_unprocessed_insert_time, max_processed_insert_time): {}. " - "This shouldn't happen often.", replica_path, Coordination::errorMessage(code)); + "This shouldn't happen often.", replica_path, code); } } @@ -551,7 +551,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / entry->znode_name); if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code)); + LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, code); updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); } @@ -1144,7 +1144,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / znode_name); if (code != Coordination::Error::ZOK) - LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), Coordination::errorMessage(code)); + LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), code); updateStateOnQueueEntryRemoval( *it, /* is_successful = */ false, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4128654a632..1e033566fed 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -723,7 +723,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: retries_ctl.setUserError( ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Insert failed due to zookeeper error. Please retry. Reason: {}", - Coordination::errorMessage(write_part_info_keeper_error)); + write_part_info_keeper_error); } retries_ctl.stopRetries(); @@ -1033,7 +1033,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: retries_ctl.setUserError( ErrorCodes::UNKNOWN_STATUS_OF_INSERT, "Unknown status, client must retry. Reason: {}", - Coordination::errorMessage(multi_code)); + multi_code); return; } else if (Coordination::isUserError(multi_code)) @@ -1109,7 +1109,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Unexpected logical error while adding block {} with ID '{}': {}, path {}", block_number, toString(block_id), - Coordination::errorMessage(multi_code), + multi_code, failed_op_path); } } @@ -1122,7 +1122,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", block_number, toString(block_id), - Coordination::errorMessage(multi_code)); + multi_code); } }, [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dac9e6923a5..06e9d88a954 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1037,7 +1037,7 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con code = zookeeper->tryMulti(ops, res); if (code != Coordination::Error::ZOK) LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (replica: {}). Will remove recursively.", - Coordination::errorMessage(code), remote_replica_path); + code, remote_replica_path); /// And finally remove everything else recursively /// It may left some garbage if replica_path subtree is concurrently modified @@ -1145,7 +1145,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper auto code = zookeeper->tryMulti(ops, res); if (code != Coordination::Error::ZOK) LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (table: {}). Will remove recursively.", - Coordination::errorMessage(code), zookeeper_path); + code, zookeeper_path); Strings children; code = zookeeper->tryGetChildren(zookeeper_path, children); @@ -1893,7 +1893,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", - entry.new_part_name, Coordination::errorMessage(code)); + entry.new_part_name, code); } else throw Coordination::Exception(code); @@ -3098,7 +3098,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (get_is_lost.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/is_lost': {}", source_replica_name, Coordination::errorMessage(get_is_lost.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/is_lost': {}", source_replica_name, get_is_lost.error); continue; } else if (get_is_lost.data != "0") @@ -3109,12 +3109,12 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (get_log_pointer.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/log_pointer': {}", source_replica_name, Coordination::errorMessage(get_log_pointer.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/log_pointer': {}", source_replica_name, get_log_pointer.error); continue; } if (get_queue.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/queue': {}", source_replica_name, Coordination::errorMessage(get_queue.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/queue': {}", source_replica_name, get_queue.error); continue; } @@ -7203,7 +7203,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( { for (size_t i = 0; i < delete_requests.size(); ++i) if (delete_responses[i]->error != Coordination::Error::ZOK) - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), Coordination::errorMessage(delete_responses[i]->error)); + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), delete_responses[i]->error); } LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", delete_requests.size(), partition_id); @@ -8717,7 +8717,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); if (code != Coordination::Error::ZOK) { - LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), code); return {true, std::nullopt}; } diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3b909194b63..3da1d5bd7b0 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,5 +1,6 @@ import socket import time +from kazoo.client import KazooClient def get_keeper_socket(cluster, node, port=9181): @@ -26,9 +27,17 @@ def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" -def wait_until_connected(cluster, node, port=9181): +def wait_until_connected(cluster, node, port=9181, timeout=30.0): + elapsed = 0.0 + while send_4lw_cmd(cluster, node, "mntr", port) == NOT_SERVING_REQUESTS_ERROR_MSG: time.sleep(0.1) + elapsed += 0.1 + + if elapsed >= timeout: + raise Exception( + f"{timeout}s timeout while waiting for {node.name} to start serving requests" + ) def wait_until_quorum_lost(cluster, node, port=9181): @@ -51,3 +60,25 @@ def get_leader(cluster, nodes): if is_leader(cluster, node): return node raise Exception("No leader in Keeper cluster.") + + +def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient: + _fake = KazooClient( + hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout + ) + _fake.start() + return _fake + + +def get_config_str(zk: KazooClient) -> str: + """ + Return decoded contents of /keeper/config node + """ + return zk.get("/keeper/config")[0].decode("utf-8") + + +def configs_equal(left: str, right: str) -> bool: + """ + Check whether /keeper/config nodes are equal + """ + return sorted(left.split("\n")) == sorted(right.split("\n")) diff --git a/tests/integration/test_keeper_nodes_move/test.py b/tests/integration/test_keeper_nodes_move/test.py index 6884ff29607..8ac7bc9b5e2 100644 --- a/tests/integration/test_keeper_nodes_move/test.py +++ b/tests/integration/test_keeper_nodes_move/test.py @@ -1,12 +1,7 @@ #!/usr/bin/env python3 - -#!/usr/bin/env python3 - import pytest from helpers.cluster import ClickHouseCluster -import random -import string import os import time from multiprocessing.dummy import Pool diff --git a/tests/integration/test_keeper_reconfig_add/__init__.py b/tests/integration/test_keeper_reconfig_add/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml new file mode 100644 index 00000000000..44e2090e9d8 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml @@ -0,0 +1,20 @@ + + + true + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml new file mode 100644 index 00000000000..e9249f7091c --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml @@ -0,0 +1,21 @@ + + + true + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + 2 node2 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml new file mode 100644 index 00000000000..a7ff1f6de28 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml @@ -0,0 +1,22 @@ + + + true + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + 2 node2 9234 + 3 node3 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py new file mode 100644 index 00000000000..c80279a0727 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -0,0 +1,155 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient +from kazoo.exceptions import BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", stay_alive=True) +node3 = cluster.add_instance("node3", stay_alive=True) + +server_join_msg = "confirms it will join" +part_of_cluster = "now this node is the part of cluster" +zk1, zk2, zk3 = None, None, None + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node2.stop_clickhouse() + node2.copy_file_to_container( + os.path.join(CONFIG_DIR, "keeper2.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + node3.stop_clickhouse() + node3.copy_file_to_container( + os.path.join(CONFIG_DIR, "keeper3.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def test_reconfig_add(started_cluster): + """ + Add a node to another node. Then add another node to two. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + print("Initial config", config) + + assert len(config.split("\n")) == 1 + assert "node1" in config + assert "node2" not in config + assert "node3" not in config + + with pytest.raises(BadArgumentsException): + # duplicate id with different endpoint + zk1.reconfig(joining="server.1=localhost:1337", leaving=None, new_members=None) + + with pytest.raises(BadArgumentsException): + # duplicate endpoint + zk1.reconfig(joining="server.8=node1:9234", leaving=None, new_members=None) + + for i in range(100): + zk1.create(f"/test_three_{i}", b"somedata") + + node2.start_clickhouse() + config, _ = zk1.reconfig( + joining="server.2=node2:9234", leaving=None, new_members=None + ) + ku.wait_until_connected(cluster, node2) + + config = config.decode("utf-8") + print("After adding 2", config) + + assert len(config.split("\n")) == 2 + assert "node1" in config + assert "node2" in config + assert "node3" not in config + + zk2 = get_fake_zk(node2) + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + for i in range(100): + assert zk2.exists(f"/test_three_{i}") is not None + zk2.create(f"/test_three_{100 + i}", b"somedata") + + # Why not both? + # One node will process add_srv request, other will pull out updated config, apply + # and return true in config update thread (without calling add_srv again) + assert node1.contains_in_log(server_join_msg) or node2.contains_in_log( + server_join_msg + ) + + assert node2.contains_in_log(part_of_cluster) + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_three_0") + + for i in range(200): + assert zk1.exists(f"/test_three_{i}") is not None + + for i in range(100): + zk2.create(f"/test_four_{i}", b"somedata") + + node3.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.3=node3:9234", leaving=None, new_members=None + ) + ku.wait_until_connected(cluster, node3) + + config = config.decode("utf-8") + print("After adding 3", config) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + + zk3 = get_fake_zk(node3) + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk3.exists(f"/test_four_{i}") is not None + zk3.create(f"/test_four_{100 + i}", b"somedata") + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + + for i in range(200): + assert zk1.exists(f"/test_four_{i}") is not None + assert zk2.exists(f"/test_four_{i}") is not None + + assert node3.contains_in_log(part_of_cluster) diff --git a/tests/integration/test_keeper_reconfig_remove/__init__.py b/tests/integration/test_keeper_reconfig_remove/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml new file mode 100644 index 00000000000..bbadc2741af --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml @@ -0,0 +1,37 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml new file mode 100644 index 00000000000..0191a522a50 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml new file mode 100644 index 00000000000..345bf402336 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml @@ -0,0 +1,37 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py new file mode 100644 index 00000000000..7f0b1ee92c6 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -0,0 +1,145 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient +from kazoo.exceptions import BadVersionException, BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) + +log_msg_removed = "has been removed from the cluster" +zk1, zk2, zk3 = None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + for conn in [zk1, zk2, zk3]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_remove_followers_from_3(started_cluster): + """ + Remove 1 follower node from cluster of 3. + Then remove another follower from two left nodes. + Check that remaining node is in standalone mode. + """ + + zk1 = get_fake_zk(node1) + config, _ = zk1.get("/keeper/config") + config = config.decode("utf-8") + print("Initial config", config) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + + with pytest.raises(BadVersionException): + zk1.reconfig(joining=None, leaving="1", new_members=None, from_config=20) + with pytest.raises(BadArgumentsException): + zk1.reconfig(joining=None, leaving=None, new_members=None) + with pytest.raises(BadArgumentsException): + # bulk reconfiguration is not supported + zk1.reconfig(joining=None, leaving=None, new_members="3") + with pytest.raises(BadArgumentsException): + zk1.reconfig(joining="1", leaving="1", new_members="3") + with pytest.raises(BadArgumentsException): + # at least one node must be left + zk1.reconfig(joining=None, leaving="1,2,3", new_members=None) + + for i in range(100): + zk1.create(f"/test_two_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"test_two_{i}") is not None + assert zk3.exists(f"test_two_{i}") is not None + + config, _ = zk1.reconfig(joining=None, leaving="3", new_members=None) + config = config.decode("utf-8") + print("After removing 3", config) + + assert len(config.split("\n")) == 2 + assert "node1" in config + assert "node2" in config + assert "node3" not in config + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + for i in range(100): + assert zk2.exists(f"test_two_{i}") is not None + zk2.create(f"/test_two_{100 + i}", b"otherdata") + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + for i in range(200): + assert zk1.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_two_0") + + assert node3.contains_in_log(log_msg_removed) + + for i in range(100): + zk2.create(f"/test_two_{200 + i}", b"otherdata") + + config, _ = zk1.reconfig(joining=None, leaving="2", new_members=None) + config = config.decode("utf-8") + + print("After removing 2", config) + assert len(config.split("\n")) == 1 + assert "node1" in config + assert "node2" not in config + assert "node3" not in config + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + for i in range(300): + assert zk1.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_two_0") + + assert not node1.contains_in_log(log_msg_removed) + assert node2.contains_in_log(log_msg_removed) + assert "Mode: standalone" in zk1.command(b"stat") diff --git a/tests/integration/test_keeper_reconfig_remove_many/__init__.py b/tests/integration/test_keeper_reconfig_remove_many/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml new file mode 100644 index 00000000000..9976169624b --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml @@ -0,0 +1,47 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml new file mode 100644 index 00000000000..edc43142464 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml @@ -0,0 +1,47 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml new file mode 100644 index 00000000000..8cebcbc0808 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml @@ -0,0 +1,47 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml new file mode 100644 index 00000000000..99ac7e53f30 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml @@ -0,0 +1,47 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml new file mode 100644 index 00000000000..92102ad486b --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml @@ -0,0 +1,47 @@ + + + 9181 + 5 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py new file mode 100644 index 00000000000..6bf477ff9c9 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -0,0 +1,149 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient, KazooState +from kazoo.exceptions import BadVersionException, BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +nodes = [ + cluster.add_instance(f"node{i}", main_configs=[f"configs/keeper{i}.xml"]) + for i in range(1, 6) +] +node1, node2, node3, node4, node5 = nodes + +log_msg_removed = "has been removed from the cluster" +zk1, zk2, zk3, zk4, zk5 = None, None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + for conn in [zk1, zk2, zk3, zk4, zk5]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_remove_2_and_leader(started_cluster): + """ + Remove 2 followers from a cluster of 5. Remove leader from 3 nodes. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + print("Initial config", config) + + assert len(config.split("\n")) == 5 + + for i in range(100): + zk1.create(f"/test_two_{i}", b"somedata") + + zk4 = get_fake_zk(node4) + zk4.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + zk5 = get_fake_zk(node5) + zk5.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk5)) + + for i in range(100): + assert zk4.exists(f"test_two_{i}") is not None + assert zk5.exists(f"test_two_{i}") is not None + + zk4.create(f"/test_two_{100 + i}", b"otherdata") + + zk2 = get_fake_zk(node2) + config, _ = zk2.reconfig(joining=None, leaving="4,5", new_members=None) + config = config.decode("utf-8") + + print("After removing 4,5", config) + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + assert "node5" not in config + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + assert ku.configs_equal(config, ku.get_config_str(zk1)) + + for i in range(200): + assert zk1.exists(f"test_two_{i}") is not None + assert zk2.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk4.stop() + zk4.close() + zk4 = get_fake_zk(node4) + zk4.sync("/test_two_0") + + with pytest.raises(Exception): + zk5.stop() + zk5.close() + zk5 = get_fake_zk(node5) + zk5.sync("/test_two_0") + + assert not node1.contains_in_log(log_msg_removed) + assert not node2.contains_in_log(log_msg_removed) + assert not node3.contains_in_log(log_msg_removed) + assert node4.contains_in_log(log_msg_removed) + assert node5.contains_in_log(log_msg_removed) + + assert ku.is_leader(cluster, node1) + + for i in range(100): + zk1.create(f"/test_leader_{i}", b"somedata") + + # when a leader gets a remove request, it must yield leadership + config, _ = zk1.reconfig(joining=None, leaving="1", new_members=None) + config = config.decode("utf-8") + print("After removing 1 (leader)", config) + + assert len(config.split("\n")) == 2 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + assert "node5" not in config + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_leader_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_leader_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"test_leader_{i}") is not None + assert zk3.exists(f"test_leader_{i}") is not None + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_leader_0") + + assert node1.contains_in_log(log_msg_removed) + assert not node2.contains_in_log(log_msg_removed) + assert not node3.contains_in_log(log_msg_removed) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml new file mode 100644 index 00000000000..71f3403aca3 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml @@ -0,0 +1,35 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml new file mode 100644 index 00000000000..faefb4d1102 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml @@ -0,0 +1,35 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml new file mode 100644 index 00000000000..80a9caa92c2 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml @@ -0,0 +1,35 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml new file mode 100644 index 00000000000..9fd88fe5d63 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml @@ -0,0 +1,21 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + 2 node2 9234 + 3 node3 9234 + 4 node4 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py new file mode 100644 index 00000000000..1b23aa056c6 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -0,0 +1,127 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from os.path import join, dirname, realpath +import time +import helpers.keeper_utils as ku +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = join(dirname(realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) +node4 = cluster.add_instance("node4", stay_alive=True) +zk1, zk2, zk3, zk4 = None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node4.stop_clickhouse() + node4.copy_file_to_container( + join(CONFIG_DIR, "keeper4.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3, zk4]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_replace_leader(started_cluster): + """ + Remove leader from a cluster of 3 and add a new node via two commands. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + for i in range(100): + zk1.create(f"/test_four_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"/test_four_{i}") is not None + assert zk3.exists(f"/test_four_{i}") is not None + + assert ku.is_leader(cluster, node1) + config, _ = zk2.reconfig(joining=None, leaving="1", new_members=None) + config = config.decode("utf-8") + + print("After removing 1 (leader)", config) + assert len(config.split("\n")) == 2 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + node4.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.4=node4:9234", leaving=None, new_members=None + ) + config = config.decode("utf-8") + ku.wait_until_connected(cluster, node4) + + print("After adding 4", config) + assert len(config.split("\n")) == 3 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" in config + + zk4 = get_fake_zk(node4) + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + for i in range(100): + assert zk4.exists(f"test_four_{i}") is not None + zk4.create(f"/test_four_{100 + i}", b"somedata") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(200): + assert zk2.exists(f"test_four_{i}") is not None + assert zk3.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml new file mode 100644 index 00000000000..71f3403aca3 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml @@ -0,0 +1,35 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml new file mode 100644 index 00000000000..faefb4d1102 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml @@ -0,0 +1,35 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml new file mode 100644 index 00000000000..80a9caa92c2 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml @@ -0,0 +1,35 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml new file mode 100644 index 00000000000..9fd88fe5d63 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml @@ -0,0 +1,21 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + 2 node2 9234 + 3 node3 9234 + 4 node4 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py new file mode 100644 index 00000000000..c7aed945097 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -0,0 +1,120 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from os.path import join, dirname, realpath +import time +import helpers.keeper_utils as ku +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = join(dirname(realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) +node4 = cluster.add_instance("node4", stay_alive=True) +zk1, zk2, zk3, zk4 = None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node4.stop_clickhouse() + node4.copy_file_to_container( + join(CONFIG_DIR, "keeper4.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3, zk4]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_replace_leader_in_one_command(started_cluster): + """ + Remove leader from a cluster of 3 and add a new node to this cluster in a single command + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + for i in range(100): + zk1.create(f"/test_four_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"/test_four_{i}") is not None + assert zk3.exists(f"/test_four_{i}") is not None + + assert ku.is_leader(cluster, node1) + node4.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.4=node4:9234", leaving="1", new_members=None + ) + config = config.decode("utf-8") + + print("After removing 1 and adding 4", config) + assert len(config.split("\n")) == 3 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" in config + + ku.wait_until_connected(cluster, node4) + time.sleep(1) + + zk4 = get_fake_zk(node4) + zk4.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + for i in range(100): + assert zk4.exists(f"test_four_{i}") is not None + zk4.create(f"/test_four_{100 + i}", b"somedata") + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(200): + assert zk2.exists(f"test_four_{i}") is not None + assert zk3.exists(f"test_four_{i}") is not None diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 5a6fd15d72c..8d685d65d1d 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -15,7 +15,7 @@ using namespace DB; void dumpMachine(std::shared_ptr machine) { - auto & storage = machine->getStorage(); + auto & storage = machine->getStorageForUnitTests(); std::queue keys; keys.push("/"); From b4f750ed66a7579580b88deaaedf15ac153785d0 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 10:11:44 +0000 Subject: [PATCH 114/473] review fixes --- base/base/find_symbols.h | 2 +- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Coordination/KeeperContext.cpp | 3 ++- src/Coordination/KeeperContext.h | 5 +++-- src/Coordination/KeeperDispatcher.cpp | 5 ++--- src/Coordination/KeeperReconfiguration.cpp | 5 ++--- src/Coordination/KeeperReconfiguration.h | 4 ++-- src/Coordination/KeeperServer.cpp | 14 +++++++------- src/Coordination/KeeperServer.h | 2 -- src/Coordination/KeeperStateMachine.cpp | 10 ++++++---- src/Coordination/KeeperStateMachine.h | 2 -- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/RaftServerConfig.cpp | 2 +- 13 files changed, 28 insertions(+), 30 deletions(-) diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index f7d24ccfc11..fda94edaa88 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end) /// See https://github.com/boostorg/algorithm/issues/63 /// And https://bugs.llvm.org/show_bug.cgi?id=41141 template -inline To& splitInto(To & to, std::string_view what, bool token_compress = false) +inline To & splitInto(To & to, std::string_view what, bool token_compress = false) { const char * pos = what.data(); const char * end = pos + what.size(); diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 20ce2a748e6..8567a53699e 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -573,7 +573,7 @@ public: template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(Coordination::Error code, auto& ctx) + constexpr auto format(Coordination::Error code, auto & ctx) { return formatter::format(Coordination::errorMessage(code), ctx); } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 3c3c0500540..0c083971f74 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -32,8 +32,9 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) { + dispatcher = dispatcher_; digest_enabled = config.getBool("keeper_server.digest_enabled", false); ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 4fb552f20a3..ba1a81b4423 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -24,7 +24,7 @@ public: SHUTDOWN }; - void initialize(const Poco::Util::AbstractConfiguration & config); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_); Phase getServerState() const; void setServerState(Phase server_state_); @@ -52,6 +52,8 @@ public: void dumpConfiguration(WriteBufferFromOwnString & buf) const; + constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -90,5 +92,4 @@ private: }; using KeeperContextPtr = std::shared_ptr; - } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 178453b2f5b..26be2881780 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -337,8 +337,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config); - keeper_context->dispatcher = this; + keeper_context->initialize(config, this); server = std::make_unique( configuration_and_settings, @@ -762,7 +761,7 @@ void KeeperDispatcher::clusterUpdateThread() } } -void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions&& actions) +void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions) { if (shutdown_called) return; for (auto && action : actions) diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index dec3e1f155f..f262a07209d 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -1,5 +1,4 @@ #include "KeeperReconfiguration.h" -#include #include #include #include @@ -45,8 +44,8 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: for (std::string_view leaving_server : leaving_arr) { - int id; - if (std::from_chars(leaving_server.begin(), leaving_server.end(), id).ec != std::error_code{}) + int32_t id; + if (!tryParse(id, leaving_server)) return {}; if (remove_ids.contains(id)) diff --git a/src/Coordination/KeeperReconfiguration.h b/src/Coordination/KeeperReconfiguration.h index 71958f2035e..fdd81708da2 100644 --- a/src/Coordination/KeeperReconfiguration.h +++ b/src/Coordination/KeeperReconfiguration.h @@ -1,6 +1,6 @@ #pragma once -#include "Coordination/KeeperSnapshotManager.h" -#include "Coordination/RaftServerConfig.h" +#include +#include namespace DB { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 1cde957ef3a..f6715b0da3f 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -793,10 +793,10 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) { std::lock_guard _{server_write_mutex}; - if (const auto* add = std::get_if(&action)) + if (const auto * add = std::get_if(&action)) return raft_instance->get_srv_config(add->id) != nullptr || raft_instance->add_srv(static_cast(*add))->get_accepted(); - else if (const auto* remove = std::get_if(&action)) + else if (const auto * remove = std::get_if(&action)) { if (isLeader() && remove->id == state_manager->server_id()) { @@ -807,7 +807,7 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) return raft_instance->get_srv_config(remove->id) == nullptr || raft_instance->remove_srv(remove->id)->get_accepted(); } - else if (const auto* update = std::get_if(&action)) + else if (const auto * update = std::get_if(&action)) { if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr) throw Exception(ErrorCodes::RAFT_ERROR, @@ -851,7 +851,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi std::this_thread::sleep_for(sleep_time * (i + 1)); }; - if (const auto* add = std::get_if(&action)) + if (const auto * add = std::get_if(&action)) { for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { @@ -863,7 +863,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi backoff_on_refusal(i); } } - else if (const auto* remove = std::get_if(&action)) + else if (const auto * remove = std::get_if(&action)) { if (remove->id == state_manager->server_id()) { @@ -884,7 +884,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi backoff_on_refusal(i); } } - else if (const auto* update = std::get_if(&action)) + else if (const auto * update = std::get_if(&action)) { raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return; @@ -928,7 +928,7 @@ bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAc backoff(i); } } - else if (std::get_if(&action) != nullptr) + else if (std::holds_alternative(action)) return true; return false; diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 61e29b67bbd..50d229c9e63 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -17,8 +17,6 @@ namespace DB using RaftAppendResult = nuraft::ptr>>; -class KeeperDispatcher; - class KeeperServer { private: diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 3e9850caa40..6ec03235a2d 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,15 +16,13 @@ #include #include #include -#include "Coordination/KeeperStorage.h" -#include "Coordination/KeeperReconfiguration.h" - #include namespace ProfileEvents { extern const Event KeeperCommits; + extern const Event KeeperReconfigRequest; extern const Event KeeperCommitsFailed; extern const Event KeeperSnapshotCreations; extern const Event KeeperSnapshotCreationsFailed; @@ -298,6 +298,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const KeeperStorage::RequestForSession& request_for_session) { + ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); + const auto& request = static_cast(*request_for_session.request); const int64_t session_id = request_for_session.session_id; const int64_t zxid = request_for_session.zxid; @@ -312,7 +314,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(res) }; }; - KeeperDispatcher& dispatcher = *keeper_context->dispatcher; + KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); if (!dispatcher.clusterUpdateQueueEmpty()) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 3b239adae45..4ff46394fcc 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,8 +12,6 @@ namespace DB { -class KeeperDispatcher; - using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 2b245a455b7..7fe85857ccb 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1088,7 +1088,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce if (request.path == Coordination::keeper_config_path) { response.data = serializeClusterConfig( - storage.keeper_context->dispatcher->getStateMachine().getClusterConfig()); + storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); response.error = Coordination::Error::ZOK; return response_ptr; } diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 42923dd0b29..45b6d5d1dad 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -31,7 +31,7 @@ std::optional RaftServerConfig::parse(std::string_view server) if (!id_str.starts_with("server.")) return std::nullopt; - int id; + Int32 id; if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) return std::nullopt; if (id <= 0) From 297d566600c3b36a552b456f8371440c5939b1d7 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 14:13:26 +0000 Subject: [PATCH 115/473] acl check --- src/Coordination/KeeperStateMachine.cpp | 3 +++ src/Coordination/KeeperStateMachine.h | 5 +++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6ec03235a2d..e053e481b6b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -314,6 +314,9 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(res) }; }; + if (!storage->checkACL(keeper_config_path, Coordination::ACL::Write, session_id, true)) + return bad_request(ZNOAUTH); + KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 4ff46394fcc..997a03a04d5 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -183,7 +183,8 @@ private: KeeperSnapshotManagerS3 * snapshot_manager_s3; - KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession& request_for_session); + KeeperStorage::ResponseForSession processReconfiguration( + const KeeperStorage::RequestForSession& request_for_session) + TSA_REQUIRES(storage_and_responses_lock); }; - } From 1bef6fc76cf70b6faeb82b25e53e708bbf309bc6 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 15:04:31 +0000 Subject: [PATCH 116/473] process reconfig in keeper dispatcher --- src/Coordination/KeeperDispatcher.cpp | 6 ++++++ src/Coordination/KeeperStateMachine.cpp | 21 +++++++++++++-------- src/Coordination/KeeperStateMachine.h | 2 ++ 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 26be2881780..9039b3a6d11 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -90,6 +90,12 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; + if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + { + server->getKeeperStateMachine()->reconfigure(request); + continue; + } + KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e053e481b6b..c837b93ffdd 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -295,6 +295,19 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } +void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) +{ + std::lock_guard _(storage_and_responses_lock); + KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); + if (!responses_queue.push(response)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); + LOG_WARNING(log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response.session_id); + } +} + KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const KeeperStorage::RequestForSession& request_for_session) { @@ -400,14 +413,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response->session_id = session_id; try_push(response_for_session); } - // Processing reconfig request as an ordinary one (in KeeperStorage) brings multiple inconsistencies - // regarding replays of old reconfigurations in new nodes. Thus the storage is not involved. - // See https://github.com/ClickHouse/ClickHouse/pull/49450 for details - else if (op_num == Coordination::OpNum::Reconfig) - { - std::lock_guard lock(storage_and_responses_lock); - try_push(processReconfiguration(*request_for_session)); - } else { if (op_num == Coordination::OpNum::Close) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 997a03a04d5..5762476886c 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -126,6 +126,8 @@ public: void recalculateStorageStats(); + void reconfigure(const KeeperStorage::RequestForSession& request_for_session); + private: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored From 4550b15876c7e57533e7aa700b1376682c95de69 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 16:38:26 +0000 Subject: [PATCH 117/473] try updating tests to wait for cluster configs to come in sync --- tests/integration/helpers/keeper_utils.py | 14 +++++++++++--- tests/integration/test_keeper_reconfig_add/test.py | 4 ++-- .../test_keeper_reconfig_remove/test.py | 6 +++--- .../test_keeper_reconfig_remove_many/test.py | 10 +++++----- .../test_keeper_reconfig_replace_leader/test.py | 10 +++++----- .../test.py | 10 +++++----- 6 files changed, 31 insertions(+), 23 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3da1d5bd7b0..3970aa325ad 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -77,8 +77,16 @@ def get_config_str(zk: KazooClient) -> str: return zk.get("/keeper/config")[0].decode("utf-8") -def configs_equal(left: str, right: str) -> bool: +def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = 30.0): """ - Check whether /keeper/config nodes are equal + Check whether get /keeper/config result in left_config is equal + to get /keeper/config on right_zk ZK connection. """ - return sorted(left.split("\n")) == sorted(right.split("\n")) + elapsed: float = 0. + while sorted(left_config.split("\n")) != sorted(get_config_str(right_zk).split("\n")): + time.sleep(1) + elapsed += 1 + if elapsed >= timeout: + raise Exception( + f"timeout while checking nodes configs to get equal. " + f"Left: {left_config}, right: {get_config_str(right_zk)}") diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py index c80279a0727..2c2da7403a1 100644 --- a/tests/integration/test_keeper_reconfig_add/test.py +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -91,7 +91,7 @@ def test_reconfig_add(started_cluster): assert "node3" not in config zk2 = get_fake_zk(node2) - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) for i in range(100): assert zk2.exists(f"/test_three_{i}") is not None @@ -132,7 +132,7 @@ def test_reconfig_add(started_cluster): assert "node3" in config zk3 = get_fake_zk(node3) - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk3.exists(f"/test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py index 7f0b1ee92c6..fb0a9472df3 100644 --- a/tests/integration/test_keeper_reconfig_remove/test.py +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -70,11 +70,11 @@ def test_reconfig_remove_followers_from_3(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"test_two_{i}") is not None @@ -92,7 +92,7 @@ def test_reconfig_remove_followers_from_3(started_cluster): zk2.stop() zk2.close() zk2 = get_fake_zk(node2) - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) for i in range(100): assert zk2.exists(f"test_two_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py index 6bf477ff9c9..ec0d8b95eff 100644 --- a/tests/integration/test_keeper_reconfig_remove_many/test.py +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -54,11 +54,11 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk4 = get_fake_zk(node4) zk4.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) zk5 = get_fake_zk(node5) zk5.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk5)) + ku.wait_configs_equal(config, zk5) for i in range(100): assert zk4.exists(f"test_two_{i}") is not None @@ -83,7 +83,7 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk1 = get_fake_zk(node1) zk1.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk1)) + ku.wait_configs_equal(config, zk1) for i in range(200): assert zk1.exists(f"test_two_{i}") is not None @@ -128,11 +128,11 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_leader_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_leader_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"test_leader_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index 1b23aa056c6..ca1ec3a0c92 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -62,11 +62,11 @@ def test_reconfig_replace_leader(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"/test_four_{i}") is not None @@ -104,7 +104,7 @@ def test_reconfig_replace_leader(started_cluster): assert "node4" in config zk4 = get_fake_zk(node4) - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) for i in range(100): assert zk4.exists(f"test_four_{i}") is not None @@ -114,13 +114,13 @@ def test_reconfig_replace_leader(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3.stop() zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index c7aed945097..76aed1c7f3a 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -62,11 +62,11 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"/test_four_{i}") is not None @@ -91,7 +91,7 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk4 = get_fake_zk(node4) zk4.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) for i in range(100): assert zk4.exists(f"test_four_{i}") is not None @@ -107,13 +107,13 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3.stop() zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None From 2f0cd054970015799b394588b7ecf79ca34a6e9a Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 17:35:25 +0000 Subject: [PATCH 118/473] handle leader removal corner cases --- src/Coordination/KeeperServer.cpp | 7 +++++-- .../test.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f6715b0da3f..fd82f220f9b 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -798,9 +798,12 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) || raft_instance->add_srv(static_cast(*add))->get_accepted(); else if (const auto * remove = std::get_if(&action)) { - if (isLeader() && remove->id == state_manager->server_id()) + if (remove->id == raft_instance->get_leader()) { - raft_instance->yield_leadership(); + if (isLeader()) + raft_instance->yield_leadership(); + else + raft_instance->request_leadership(); return false; } diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 76aed1c7f3a..e23d0674c12 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -113,7 +113,7 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - ku.configs_equal(config, zk3) + ku.wait_configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None From c2a0607cf890f95e94db2751a3e68b7acc59a5bf Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 17:52:57 +0000 Subject: [PATCH 119/473] fix --- tests/integration/helpers/keeper_utils.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3970aa325ad..93ea3fa74b7 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -82,11 +82,14 @@ def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = Check whether get /keeper/config result in left_config is equal to get /keeper/config on right_zk ZK connection. """ - elapsed: float = 0. - while sorted(left_config.split("\n")) != sorted(get_config_str(right_zk).split("\n")): + elapsed: float = 0.0 + while sorted(left_config.split("\n")) != sorted( + get_config_str(right_zk).split("\n") + ): time.sleep(1) elapsed += 1 if elapsed >= timeout: raise Exception( f"timeout while checking nodes configs to get equal. " - f"Left: {left_config}, right: {get_config_str(right_zk)}") + f"Left: {left_config}, right: {get_config_str(right_zk)}" + ) From c46b125d0a8501241a4a726a32141e1215a2cbf5 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 5 Jul 2023 18:23:34 +0000 Subject: [PATCH 120/473] review fixes --- src/Coordination/KeeperDispatcher.cpp | 2 ++ src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateMachine.h | 5 ++++- src/Coordination/tests/gtest_coordination.cpp | 10 +++++----- utils/keeper-data-dumper/main.cpp | 2 +- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9039b3a6d11..90996dfaff7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -121,6 +121,8 @@ void KeeperDispatcher::requestThread() std::lock_guard lock(read_request_queue_mutex); read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } + else if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + server->getKeeperStateMachine()->reconfigure(request); else { current_batch_bytes_size += request.request->bytesSize(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c837b93ffdd..45c776e105b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -313,7 +313,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); - const auto& request = static_cast(*request_for_session.request); + const auto & request = static_cast(*request_for_session.request); const int64_t session_id = request_for_session.session_id; const int64_t zxid = request_for_session.zxid; diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 5762476886c..116fa9257a0 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -88,7 +88,10 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - KeeperStorage & getStorageForUnitTests() TSA_NO_THREAD_SAFETY_ANALYSIS + // This should be used only for tests or keeper-data-dumper because it violates + // TSA -- we can't acquire the lock outside of this class or return a storage under lock + // in a reasonable way. + KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS { return *storage; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index b302f9b13ca..03ce23e9233 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1616,8 +1616,8 @@ void testLogAndStateMachine( restore_machine->commit(i, changelog.entry_at(i)->get_buf()); } - auto & source_storage = state_machine->getStorageForUnitTests(); - auto & restored_storage = restore_machine->getStorageForUnitTests(); + auto & source_storage = state_machine->getStorageUnsafe(); + auto & restored_storage = restore_machine->getStorageUnsafe(); EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1719,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); state_machine->pre_commit(1, entry_c->get_buf()); state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorageForUnitTests(); + const auto & storage = state_machine->getStorageUnsafe(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); @@ -1768,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); state_machine->pre_commit(2, create_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); // commit log entries @@ -1831,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) state_machine->commit(2, create_entry->get_buf()); state_machine->commit(3, set_acl_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; auto node = uncommitted_state.getNode(node_path); ASSERT_NE(node, nullptr); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 8d685d65d1d..22e5f47687a 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -15,7 +15,7 @@ using namespace DB; void dumpMachine(std::shared_ptr machine) { - auto & storage = machine->getStorageForUnitTests(); + auto & storage = machine->getStorageUnsafe(); std::queue keys; keys.push("/"); From bafcc3afdc79463915b53b5e441758cbb958b958 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 6 Jul 2023 15:18:49 +0000 Subject: [PATCH 121/473] remove reconfig in process flag as it's useless --- src/Common/ZooKeeper/IKeeper.h | 1 - src/Coordination/KeeperDispatcher.cpp | 5 ----- src/Coordination/KeeperDispatcher.h | 1 - src/Coordination/KeeperStateMachine.cpp | 4 +--- src/Coordination/RaftServerConfig.cpp | 9 ++++----- 5 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 8567a53699e..5240acc2616 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -82,7 +82,6 @@ enum class Error : int32_t ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments ZINVALIDSTATE = -9, /// Invalid zhandle state - ZRECONFIGINPROGRESS = -14, /// Another reconfig is running /** API errors. * This is never thrown by the server, it shouldn't be used other than diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 90996dfaff7..b956bba4031 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -780,11 +780,6 @@ void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions) } } -bool KeeperDispatcher::clusterUpdateQueueEmpty() const -{ - return cluster_update_queue.empty(); -} - bool KeeperDispatcher::reconfigEnabled() const { return server->reconfigEnabled(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index a9b3d33eb51..40f1dac1570 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -136,7 +136,6 @@ public: void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void pushClusterUpdates(ClusterUpdateActions&& actions); - bool clusterUpdateQueueEmpty() const; bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 45c776e105b..b821050cccf 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -309,7 +309,7 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req } KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( - const KeeperStorage::RequestForSession& request_for_session) + const KeeperStorage::RequestForSession & request_for_session) { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); @@ -333,8 +333,6 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); - if (!dispatcher.clusterUpdateQueueEmpty()) - return bad_request(ZRECONFIGINPROGRESS); if (request.version != -1) return bad_request(ZBADVERSION); diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 45b6d5d1dad..9090ed68fb6 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -1,7 +1,6 @@ #include "RaftServerConfig.h" -#include -#include #include +#include #include namespace DB @@ -32,7 +31,7 @@ std::optional RaftServerConfig::parse(std::string_view server) return std::nullopt; Int32 id; - if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) + if (!tryParse(id, std::next(id_str.begin(), 7))) return std::nullopt; if (id <= 0) return std::nullopt; @@ -44,7 +43,7 @@ std::optional RaftServerConfig::parse(std::string_view server) const std::string_view port = endpoint.substr(port_delimiter + 1); uint16_t port_tmp; - if (std::from_chars(port.begin(), port.end(), port_tmp).ec != std::error_code{}) + if (!tryParse(port_tmp, port)) return std::nullopt; RaftServerConfig out{id, endpoint}; @@ -59,7 +58,7 @@ std::optional RaftServerConfig::parse(std::string_view server) return out; const std::string_view priority = parts[3]; - if (std::from_chars(priority.begin(), priority.end(), out.priority).ec != std::error_code{}) + if (!tryParse(out.priority, priority)) return std::nullopt; if (out.priority < 0) return std::nullopt; From 5302b478a4b512d080068563d4b5b983e4b13d77 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 6 Jul 2023 17:12:24 +0000 Subject: [PATCH 122/473] proper reconfig batch handling --- src/Common/ZooKeeper/IKeeper.cpp | 1 - src/Coordination/KeeperDispatcher.cpp | 32 +++++++++++++++++---------- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 50160279506..f0a07241735 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -110,7 +110,6 @@ const char * errorMessage(Error code) case Error::ZCLOSING: return "ZooKeeper is closing"; case Error::ZNOTHING: return "(not error) no server responses to process"; case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; - case Error::ZRECONFIGINPROGRESS: return "Another reconfiguration is progress"; } UNREACHABLE(); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b956bba4031..daa65de0d89 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -82,6 +82,7 @@ void KeeperDispatcher::requestThread() /// requests into a batch we must check that the new request is not read request. Otherwise we have to /// process all already accumulated write requests, wait them synchronously and only after that process /// read request. So reads are some kind of "separator" for writes. + /// Also there is a special reconfig request also being a separator. try { if (requests_queue->tryPop(request, max_wait)) @@ -90,20 +91,17 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - if (request.request->getOpNum() == Coordination::OpNum::Reconfig) - { - server->getKeeperStateMachine()->reconfigure(request); - continue; - } - KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; bool has_read_request = false; + bool has_reconfig_request = false; - /// If new request is not read request or we must to process it through quorum. + /// If new request is not read request or reconfig request we must process it through quorum. /// Otherwise we will process it locally. - if (coordination_settings->quorum_reads || !request.request->isReadRequest()) + if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + has_reconfig_request = true; + else if (coordination_settings->quorum_reads || !request.request->isReadRequest()) { current_batch_bytes_size += request.request->bytesSize(); current_batch.emplace_back(request); @@ -122,7 +120,10 @@ void KeeperDispatcher::requestThread() read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } else if (request.request->getOpNum() == Coordination::OpNum::Reconfig) - server->getKeeperStateMachine()->reconfigure(request); + { + has_reconfig_request = true; + return false; + } else { current_batch_bytes_size += request.request->bytesSize(); @@ -138,6 +139,7 @@ void KeeperDispatcher::requestThread() /// TODO: Deprecate max_requests_quick_batch_size and use only max_requests_batch_size and max_requests_batch_bytes_size size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size; while (!shutdown_called && !has_read_request && + !has_reconfig_request && current_batch.size() < max_quick_batch_size && current_batch_bytes_size < max_batch_bytes_size && try_get_request()) ; @@ -150,8 +152,10 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_read_request && !prev_result_done() && - current_batch.size() <= max_batch_size && current_batch_bytes_size < max_batch_bytes_size) + while (!shutdown_called && !has_read_request && + !has_reconfig_request && !prev_result_done() && + current_batch.size() <= max_batch_size + && current_batch_bytes_size < max_batch_bytes_size) { try_get_request(); } @@ -175,7 +179,8 @@ void KeeperDispatcher::requestThread() if (result) { - if (has_read_request) /// If we will execute read request next, than we have to process result now + /// If we will execute read or reconfig next, we have to process result now + if (has_read_request || has_reconfig_request) forceWaitAndProcessResult(result, current_batch); } else @@ -189,6 +194,9 @@ void KeeperDispatcher::requestThread() prev_result = result; } + if (has_reconfig_request) + server->getKeeperStateMachine()->reconfigure(request); + /// Read request always goes after write batch (last request) if (has_read_request) { From acd17c7974637714138a76fb83f73ec31946aa79 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 10:40:04 +0800 Subject: [PATCH 123/473] Make a deal with the "Style check" --- src/Loggers/Loggers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 1e169190ca4..4c85ea79a63 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -39,7 +39,7 @@ static std::string renderFileNameTemplate(time_t now, const std::string & file_p fs::path path{file_path}; std::tm buf; localtime_r(&now, &buf); - std::stringstream ss; + std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss << std::put_time(&buf, file_path.c_str()); return path.replace_filename(ss.str()); } From 7080d85d2de6c743cc5759fa2a50d1ada1d51068 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 10:54:36 +0800 Subject: [PATCH 124/473] Amend the documentation --- .../settings.md | 49 +++++++++++++++++- .../settings.md | 50 ++++++++++++++++++- 2 files changed, 95 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bad7e388377..48361b0f157 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1201,13 +1201,58 @@ Keys: - `console` – Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`. - `stream_compress` – Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`. +Both log and error log file names (only file names, not directories) support date and time format placeholders. + +**Placeholders** +Using the following placeholders, you can define a pattern for the resulting file name. “Example” column shows formatting result for `2023-07-06 18:32:07`. + +| Placeholder | Description | Example | +|-------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| +| %% | Literal % | % | +| %n | New-line character | | +| %t | Horizontal tab character | | +| %Y | Year as a decimal number, e.g. 2017 | 2023 | +| %y | Last 2 digits of year as a decimal number (range [00,99]) | 23 | +| %C | First 2 digits of year as a decimal number (range [00,99]) | 20 | +| %G | Four-digit [ISO 8601 week-based year](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), i.e. the year that contains the specified week. Normally useful only with %V | 2023 | +| %g | Last 2 digits of [ISO 8601 week-based year](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), i.e. the year that contains the specified week. | 23 | +| %b | Abbreviated month name, e.g. Oct (locale dependent) | Jul | +| %h | Synonym of %b | Jul | +| %B | Full month name, e.g. October (locale dependent) | July | +| %m | Month as a decimal number (range [01,12]) | 07 | +| %U | Week of the year as a decimal number (Sunday is the first day of the week) (range [00,53]) | 27 | +| %W | Week of the year as a decimal number (Monday is the first day of the week) (range [00,53]) | 27 | +| %V | ISO 8601 week number (range [01,53]) | 27 | +| %j | Day of the year as a decimal number (range [001,366]) | 187 | +| %d | Day of the month as a zero-padded decimal number (range [01,31]). Single digit is preceded by zero. | 06 | +| %e | Day of the month as a space-padded decimal number (range [1,31]). Single digit is preceded by a space. |   6 | +| %a | Abbreviated weekday name, e.g. Fri (locale dependent) | Thu | +| %A | Full weekday name, e.g. Friday (locale dependent) | Thursday | +| %w | Weekday as a integer number with Sunday as 0 (range [0-6]) | 4 | +| %u | Weekday as a decimal number, where Monday is 1 (ISO 8601 format) (range [1-7]) | 4 | +| %H | Hour as a decimal number, 24 hour clock (range [00-23]) | 18 | +| %I | Hour as a decimal number, 12 hour clock (range [01,12]) | 06 | +| %M | Minute as a decimal number (range [00,59]) | 32 | +| %S | Second as a decimal number (range [00,60]) | 07 | +| %c | Standard date and time string, e.g. Sun Oct 17 04:41:13 2010 (locale dependent) | Thu Jul 6 18:32:07 2023 | +| %x | Localized date representation (locale dependent) | 07/06/23 | +| %X | Localized time representation, e.g. 18:40:20 or 6:40:20 PM (locale dependent) | 18:32:07 | +| %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 07/06/23 | +| %F | Short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2023-07-06 | +| %r | Localized 12-hour clock time (locale dependent) | 06:32:07 PM | +| %R | Equivalent to "%H:%M" | 18:32 | +| %T | Equivalent to "%H:%M:%S" (the ISO 8601 time format) | 18:32:07 | +| %p | Localized a.m. or p.m. designation (locale dependent) | PM | +| %z | Offset from UTC in the ISO 8601 format (e.g. -0430), or no characters if the time zone information is not available | +0800 | +| %Z | Locale-dependent time zone name or abbreviation, or no characters if the time zone information is not available | Z AWST | + **Example** ``` xml trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.err.log 1000M 10 true diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 5430469ea18..421df3fe3eb 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -575,14 +575,60 @@ ClickHouse поддерживает динамическое изменение - `errorlog` - Файл лога ошибок. - `size` - Размер файла. Действует для `log` и `errorlog`. Как только файл достиг размера `size`, ClickHouse архивирует и переименовывает его, а на его месте создает новый файл лога. - `count` - Количество заархивированных файлов логов, которые сохраняет ClickHouse. +- `stream_compress` – Сжимать `log` и `errorlog` с помощью алгоритма `lz4`. Чтобы активировать, узтановите значение `1` или `true`. + +Имена файлов `log` и `errorlog` (только имя файла, а не директорий) поддерживают спецификаторы шаблонов даты и времени. + +**Спецификаторы форматирования** +С помощью следующих спецификаторов, можно определить шаблон для формирования имени файла. Столбец “Пример” показывает возможные значения на момент времени `2023-07-06 18:32:07`. + +| Спецификатор | Описание | Пример | +|--------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| +| %% | Литерал % | % | +| %n | Символ новой строки | | +| %t | Символ горизонтальной табуляции | | +| %Y | Год как десятичное число, например, 2017 | 2023 | +| %y | Последние 2 цифры года в виде десятичного числа (диапазон [00,99]) | 23 | +| %C | Первые 2 цифры года в виде десятичного числа (диапазон [00,99]) | 20 | +| %G | Год по неделям согласно [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), то есть год, который содержит указанную неделю. Обычно используется вместе с %V. | 2023 | +| %g | Последние 2 цифры [года по неделям ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), т.е. года, содержащего указанную неделю (диапазон [00,99]). | 23 | +| %b | Сокращённое название месяца, например Oct (зависит от локали) | Jul | +| %h | Синоним %b | Jul | +| %B | Полное название месяца, например, October (зависит от локали) | July | +| %m | Месяц в виде десятичного числа (диапазон [01,12]) | 07 | +| %U | Неделя года в виде десятичного числа (воскресенье - первый день недели) (диапазон [00,53]) | 27 | +| %W | Неделя года в виде десятичного числа (понедельник - первый день недели) (диапазон [00,53]) | 27 | +| %V | Неделя года ISO 8601 (диапазон [01,53]) | 27 | +| %j | День года в виде десятичного числа (диапазон [001,366]) | 187 | +| %d | День месяца в виде десятичного числа (диапазон [01,31]) Перед одиночной цифрой ставится ноль. | 06 | +| %e | День месяца в виде десятичного числа (диапазон [1,31]). Перед одиночной цифрой ставится пробел. |   6 | +| %a | Сокращённое название дня недели, например, Fri (зависит от локали) | Thu | +| %A | Полный день недели, например, Friday (зависит от локали) | Thursday | +| %w | День недели в виде десятичного числа, где воскресенье равно 0 (диапазон [0-6]) | 4 | +| %u | День недели в виде десятичного числа, где понедельник равен 1 (формат ISO 8601) (диапазон [1-7]) | 4 | +| %H | Час в виде десятичного числа, 24-часовой формат (диапазон [00-23]) | 18 | +| %I | Час в виде десятичного числа, 12-часовой формат (диапазон [01,12]) | 06 | +| %M | Минуты в виде десятичного числа (диапазон [00,59]) | 32 | +| %S | Секунды как десятичное число (диапазон [00,60]) | 07 | +| %c | Стандартная строка даты и времени, например, Sun Oct 17 04:41:13 2010 (зависит от локали) | Thu Jul 6 18:32:07 2023 | +| %x | Локализованное представление даты (зависит от локали) | 07/06/23 | +| %X | Локализованное представление времени, например, 18:40:20 или 6:40:20 PM (зависит от локали) | 18:32:07 | +| %D | Эквивалентно "%m/%d/%y" | 07/06/23 | +| %F | Эквивалентно "%Y-%m-%d" (формат даты ISO 8601) | 2023-07-06 | +| %r | Локализованное 12-часовое время (зависит от локали) | 06:32:07 PM | +| %R | Эквивалентно "%H:%M" | 18:32 | +| %T | Эквивалентно "%H:%M:%S" (формат времени ISO 8601) | 18:32:07 | +| %p | Локализованное обозначение a.m. или p.m. (зависит от локали) | PM | +| %z | Смещение от UTC в формате ISO 8601 (например, -0430), или без символов, если информация о часовом поясе недоступна | +0800 | +| %Z | Зависящее от локали название или аббревиатура часового пояса, если информация о часовом поясе доступна | Z AWST | **Пример** ``` xml trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.err.log 1000M 10 From 63fbde41fee5fb8c0133dc5a576ed4e3caa5c3f2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 11:01:39 +0800 Subject: [PATCH 125/473] Reformat cluster.py (add empty line) --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5b583b865de..d4b1ee76712 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -72,6 +72,7 @@ CLICKHOUSE_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.log" CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.log" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") From 3edee4174c040b079015ce6524c0d4c56926e348 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 11:34:03 +0800 Subject: [PATCH 126/473] Add AWST time zone abbreviation to the ignore list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..6ddca6db538 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -12,6 +12,7 @@ ARMv ASLR ASOF ASan +AWST Actian ActionsMenu ActiveRecord From 87ea1b6667ed9a79272e3b77c529369f2acc4e4e Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 13:01:30 +0800 Subject: [PATCH 127/473] Recover the integration test --- .../test.py | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py index e69de29bb2d..9fa87056d2c 100644 --- a/tests/integration/test_render_log_file_name_templates/test.py +++ b/tests/integration/test_render_log_file_name_templates/test.py @@ -0,0 +1,58 @@ +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from datetime import datetime + + +log_dir = "/var/log/clickhouse-server/" +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.add_instance( + "file-names-from-config", + main_configs=["configs/config-file-template.xml"], + clickhouse_log_file=None, + clickhouse_error_log_file=None, + ) + cluster.add_instance( + "file-names-from-params", + clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", + clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", + ) + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_file_names(started_cluster): + now = datetime.now() + log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" + ) + err_log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" + ) + logging.debug(f"log_file {log_file} err_log_file {err_log_file}") + + for name, instance in started_cluster.instances.items(): + files = instance.exec_in_container( + ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True + ) + + logging.debug(f"check instance '{name}': {log_dir} contains: {files}") + + assert ( + instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) + == log_file + "\n" + ) + + assert ( + instance.exec_in_container( + ["bash", "-c", f"ls {err_log_file}"], nothrow=True + ) + == err_log_file + "\n" + ) From 95fedaedff3ad3e3cdb15d3cc2b06ab6d9ea1e9b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 15:16:10 +0800 Subject: [PATCH 128/473] Refine the integration test code --- .../__init__.py | 58 ------------------- .../test.py | 8 +-- 2 files changed, 2 insertions(+), 64 deletions(-) diff --git a/tests/integration/test_render_log_file_name_templates/__init__.py b/tests/integration/test_render_log_file_name_templates/__init__.py index 9fa87056d2c..e69de29bb2d 100644 --- a/tests/integration/test_render_log_file_name_templates/__init__.py +++ b/tests/integration/test_render_log_file_name_templates/__init__.py @@ -1,58 +0,0 @@ -import pytest -import logging -from helpers.cluster import ClickHouseCluster -from datetime import datetime - - -log_dir = "/var/log/clickhouse-server/" -cluster = ClickHouseCluster(__file__) - - -@pytest.fixture(scope="module") -def started_cluster(): - cluster.add_instance( - "file-names-from-config", - main_configs=["configs/config-file-template.xml"], - clickhouse_log_file=None, - clickhouse_error_log_file=None, - ) - cluster.add_instance( - "file-names-from-params", - clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", - clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", - ) - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_check_file_names(started_cluster): - now = datetime.now() - log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" - ) - err_log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" - ) - logging.debug(f"log_file {log_file} err_log_file {err_log_file}") - - for name, instance in started_cluster.instances.items(): - files = instance.exec_in_container( - ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True - ) - - logging.debug(f"check instance '{name}': {log_dir} contains: {files}") - - assert ( - instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) - == log_file + "\n" - ) - - assert ( - instance.exec_in_container( - ["bash", "-c", f"ls {err_log_file}"], nothrow=True - ) - == err_log_file + "\n" - ) diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py index 9fa87056d2c..58df32b823e 100644 --- a/tests/integration/test_render_log_file_name_templates/test.py +++ b/tests/integration/test_render_log_file_name_templates/test.py @@ -30,12 +30,8 @@ def started_cluster(): def test_check_file_names(started_cluster): now = datetime.now() - log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" - ) - err_log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" - ) + log_file = log_dir + f"clickhouse-server-{now.strftime('%Y-%m')}.log" + err_log_file = log_dir + f"clickhouse-server-{now.strftime('%Y-%m')}.err.log" logging.debug(f"log_file {log_file} err_log_file {err_log_file}") for name, instance in started_cluster.instances.items(): From 5b102ce7d44d678a674d29e4140a60950c69f537 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 17:21:47 +0800 Subject: [PATCH 129/473] Amend English version of settings.md --- .../server-configuration-parameters/settings.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 48361b0f157..82dac74e647 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1201,12 +1201,12 @@ Keys: - `console` – Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`. - `stream_compress` – Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`. -Both log and error log file names (only file names, not directories) support date and time format placeholders. +Both log and error log file names (only file names, not directories) support date and time format specifiers. -**Placeholders** -Using the following placeholders, you can define a pattern for the resulting file name. “Example” column shows formatting result for `2023-07-06 18:32:07`. +**Format specifiers** +Using the following format specifiers, you can define a pattern for the resulting file name. “Example” column shows possible results for `2023-07-06 18:32:07`. -| Placeholder | Description | Example | +| Specifier | Description | Example | |-------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| | %% | Literal % | % | | %n | New-line character | | From cd3080428ea3da6a71169c929e959a0c3f9c5d5b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:58:01 +0000 Subject: [PATCH 130/473] Fix async connect to hosts with multiple ips --- src/Client/Connection.cpp | 2 + src/Client/Connection.h | 4 ++ src/Client/ConnectionEstablisher.cpp | 7 +- src/Client/ConnectionEstablisher.h | 2 + .../configs/enable_hedged.xml | 8 +++ .../configs/listen_host.xml | 4 ++ .../test.py | 65 +++++++++++++++++++ 7 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml create mode 100644 tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml create mode 100644 tests/integration/test_async_connect_to_multiple_ips/test.py diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index cd102f46ffe..cac5600fbcb 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -105,6 +105,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) for (auto it = addresses.begin(); it != addresses.end();) { + have_more_addresses_to_connect = it != std::prev(addresses.end()); + if (connected) disconnect(); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index cb3f2507cb9..f4daf8e3aeb 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -159,6 +159,8 @@ public: out->setAsyncCallback(async_callback); } + bool haveMoreAddressesToConnect() const { return have_more_addresses_to_connect; } + private: String host; UInt16 port; @@ -227,6 +229,8 @@ private: std::shared_ptr maybe_compressed_out; std::unique_ptr block_out; + bool have_more_addresses_to_connect = false; + /// Logger is created lazily, for avoid to run DNS request in constructor. class LoggerWrapper { diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 897fb5fde73..439025447ca 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -179,7 +179,7 @@ bool ConnectionEstablisherAsync::checkTimeout() is_timeout_alarmed = true; } - if (is_timeout_alarmed && !is_socket_ready) + if (is_timeout_alarmed && !is_socket_ready && !haveMoreAddressesToConnect()) { /// In not async case timeout exception would be thrown and caught in ConnectionEstablisher::run, /// but in async case we process timeout outside and cannot throw exception. So, we just save fail message. @@ -225,6 +225,11 @@ void ConnectionEstablisherAsync::resetResult() } } +bool ConnectionEstablisherAsync::haveMoreAddressesToConnect() +{ + return !result.entry.isNull() && result.entry->haveMoreAddressesToConnect(); +} + #endif } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 5b58563dc01..a8126900d3b 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -104,6 +104,8 @@ private: void resetResult(); + bool haveMoreAddressesToConnect(); + ConnectionEstablisher connection_establisher; TryResult result; std::string fail_message; diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml new file mode 100644 index 00000000000..238370176af --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml @@ -0,0 +1,8 @@ + + + + 1 + 0 + + + diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml new file mode 100644 index 00000000000..df0247fd651 --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml @@ -0,0 +1,4 @@ + + :: + + diff --git a/tests/integration/test_async_connect_to_multiple_ips/test.py b/tests/integration/test_async_connect_to_multiple_ips/test.py new file mode 100644 index 00000000000..0c18a316d4b --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/test.py @@ -0,0 +1,65 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def cluster_without_dns_cache_update(): + try: + cluster.start() + + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + pass + + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/listen_host.xml"], + user_configs=["configs/enable_hedged.xml"], + with_zookeeper=True, + ipv4_address="10.5.95.11", +) + +node2 = cluster.add_instance( + "node2", + main_configs=["configs/listen_host.xml"], + user_configs=["configs/enable_hedged.xml"], + with_zookeeper=True, + ipv4_address="10.5.95.12", +) + +# node1 - source with table, have invalid ipv6 +# node2 - destination, doing remote query +def test(cluster_without_dns_cache_update): + node1.query( + "CREATE TABLE test(t Date, label UInt8) ENGINE = MergeTree PARTITION BY t ORDER BY label;" + ) + node1.query( + "INSERT INTO test SELECT toDate('2022-12-28'), 1;" + ) + assert node1.query( + 'SELECT count(*) FROM test' + ) == '1\n' + + wrong_ip = '2001:3984:3989::1:1118' + + node2.exec_in_container( + (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(wrong_ip, node1.name)]) + ) + node2.exec_in_container( + (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name)]) + ) + + assert node1.query("SELECT count(*) from test") == "1\n" + node2.query("SYSTEM DROP DNS CACHE") + node1.query("SYSTEM DROP DNS CACHE") + assert node2.query(f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;") == "1\n" + From fc94cc8b87fceb8b6631b72e34a6c10fdc197f83 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:59:27 +0000 Subject: [PATCH 131/473] Update config for test --- .../test_async_connect_to_multiple_ips/configs/enable_hedged.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml index 238370176af..399d886ee6a 100644 --- a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml @@ -2,7 +2,6 @@ 1 - 0 From ed37b01b515ea204223dd03cee5482ee6faad351 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 14:39:11 +0000 Subject: [PATCH 132/473] Fix style --- .../__init__.py | 0 .../test.py | 29 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_async_connect_to_multiple_ips/__init__.py diff --git a/tests/integration/test_async_connect_to_multiple_ips/__init__.py b/tests/integration/test_async_connect_to_multiple_ips/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_async_connect_to_multiple_ips/test.py b/tests/integration/test_async_connect_to_multiple_ips/test.py index 0c18a316d4b..acc4d24d0fa 100644 --- a/tests/integration/test_async_connect_to_multiple_ips/test.py +++ b/tests/integration/test_async_connect_to_multiple_ips/test.py @@ -36,30 +36,37 @@ node2 = cluster.add_instance( ipv4_address="10.5.95.12", ) + # node1 - source with table, have invalid ipv6 # node2 - destination, doing remote query def test(cluster_without_dns_cache_update): node1.query( "CREATE TABLE test(t Date, label UInt8) ENGINE = MergeTree PARTITION BY t ORDER BY label;" ) - node1.query( - "INSERT INTO test SELECT toDate('2022-12-28'), 1;" - ) - assert node1.query( - 'SELECT count(*) FROM test' - ) == '1\n' - - wrong_ip = '2001:3984:3989::1:1118' + node1.query("INSERT INTO test SELECT toDate('2022-12-28'), 1;") + assert node1.query("SELECT count(*) FROM test") == "1\n" + + wrong_ip = "2001:3984:3989::1:1118" node2.exec_in_container( (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(wrong_ip, node1.name)]) ) node2.exec_in_container( - (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name)]) + ( + [ + "bash", + "-c", + "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name), + ] + ) ) assert node1.query("SELECT count(*) from test") == "1\n" node2.query("SYSTEM DROP DNS CACHE") node1.query("SYSTEM DROP DNS CACHE") - assert node2.query(f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;") == "1\n" - + assert ( + node2.query( + f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;" + ) + == "1\n" + ) From de0837fe21d3f7330fc58712c887b1cc570af05a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 03:55:25 +0200 Subject: [PATCH 133/473] Check that functional tests cleanup their tables --- tests/clickhouse-test | 15 ++++++++++++++- .../02788_fix_logical_error_in_sorting.sql | 6 ++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 95470f77987..46ec19b041d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1212,7 +1212,20 @@ class TestCase: seconds_left = max( args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - drop_database_query = "DROP DATABASE IF EXISTS " + database + + leftover_tables = clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ).decode().replace("\n", ", "); + + if 0 != len(leftover_tables): + raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + + drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: drop_database_query += " ON CLUSTER test_cluster_database_replicated" diff --git a/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql b/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql index 1ff68191800..60905e2634f 100644 --- a/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql +++ b/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql @@ -1,3 +1,6 @@ +DROP TABLE IF EXISTS session_events; +DROP TABLE IF EXISTS event_types; + CREATE TABLE session_events ( clientId UInt64, @@ -75,3 +78,6 @@ FROM WHERE runningDifference(timestamp) >= 500 ORDER BY timestamp ASC FORMAT Null; + +DROP TABLE session_events; +DROP TABLE event_types; From a61bc7cfa593a510f96c670ef987129a12dc9b40 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 9 Jul 2023 02:03:47 +0000 Subject: [PATCH 134/473] Automatic style fix --- tests/clickhouse-test | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 46ec19b041d..1145dfa9358 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1213,17 +1213,23 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - leftover_tables = clickhouse_execute( - args, - f"SHOW TABLES FROM {database}", - timeout=seconds_left, - settings={ - "log_comment": args.testcase_basename, - }, - ).decode().replace("\n", ", "); + leftover_tables = ( + clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ) + .decode() + .replace("\n", ", ") + ) if 0 != len(leftover_tables): - raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + raise Exception( + f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally." + ) drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: From 7311469c32f448a95e046c013f3ef34a2bd880b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 04:17:15 +0200 Subject: [PATCH 135/473] Fix test_extreme_deduplication --- .../test_extreme_deduplication/test.py | 27 +++++++++++-------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 71f783d37c9..9ce3582a826 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -49,20 +49,25 @@ def test_deduplication_window_in_seconds(started_cluster): node.query("INSERT INTO simple VALUES (0, 1)") assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n") - # wait clean thread - time.sleep(2) + # Wait for the cleanup thread. + for i in range(100): + time.sleep(1) + + if ( + TSV.toMat( + node.query( + "SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/tables/0/simple/blocks'" + ) + )[0][0] + <= "1" + ): + break + else: + raise Exception("The blocks from Keeper were not removed in time") - assert ( - TSV.toMat( - node.query( - "SELECT count() FROM system.zookeeper WHERE path='/clickhouse/tables/0/simple/blocks'" - ) - )[0][0] - == "1" - ) node.query( "INSERT INTO simple VALUES (0, 0)" - ) # deduplication doesn't works here, the first hash node was deleted + ) # Deduplication doesn't work here as the first hash node was deleted assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3" node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") From 00cf66ab7a60025c3722044ee6bf10235e15333e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 26 Jun 2023 21:25:43 +0200 Subject: [PATCH 136/473] Add ability to disable native copy for BACKUP/RESTORE Native copy uses CopyObject for S3/GCS, but in GCS the CopyObject is buggy - it does not always works. Signed-off-by: Azat Khuzhin --- src/Backups/BackupFactory.h | 1 + src/Backups/BackupIO_S3.cpp | 10 +++-- src/Backups/BackupIO_S3.h | 6 ++- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 ++ src/Backups/BackupsWorker.cpp | 2 + src/Backups/RestoreSettings.cpp | 1 + src/Backups/RestoreSettings.h | 3 ++ src/Backups/registerBackupEngineS3.cpp | 4 +- tests/config/config.d/storage_conf.xml | 8 ++++ .../02801_backup_native_copy.reference | 4 ++ .../0_stateless/02801_backup_native_copy.sh | 43 +++++++++++++++++++ 12 files changed, 78 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02801_backup_native_copy.reference create mode 100755 tests/queries/0_stateless/02801_backup_native_copy.sh diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 92a5e16533c..642f5cb07b9 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -35,6 +35,7 @@ public: std::shared_ptr backup_coordination; std::optional backup_uuid; bool deduplicate_files = true; + bool native_copy = true; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 9a2a457e13e..01e6bc78949 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,12 +101,13 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} + , native_copy(native_copy_) { request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } @@ -138,7 +139,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (destination_data_source_description.sameKind(data_source_description) + if (native_copy && destination_data_source_description.sameKind(data_source_description) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { /// Use native copy, the more optimal way. @@ -177,12 +178,13 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} + , native_copy(native_copy_) { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint @@ -194,7 +196,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + if (native_copy && source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index cca56bae6bc..d02e45370f9 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -32,13 +32,14 @@ private: const std::shared_ptr client; S3Settings::RequestSettings request_settings; const DataSourceDescription data_source_description; + const bool native_copy; }; class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; @@ -61,6 +62,7 @@ private: S3Settings::RequestSettings request_settings; std::optional supports_batch_delete; const DataSourceDescription data_source_description; + const bool native_copy; }; } diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 882342467fe..8e9fe7956f9 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes M(Bool, async) \ M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ + M(Bool, native_copy) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 2c899687e6e..e21b70ee25f 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -38,6 +38,9 @@ struct BackupSettings /// Whether the BACKUP will omit similar files (within one backup only). bool deduplicate_files = true; + /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) + bool native_copy = true; + /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. size_t shard_num = 0; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 53a076705c4..fddd4f34bb6 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -348,6 +348,7 @@ void BackupsWorker::doBackup( backup_create_params.backup_coordination = backup_coordination; backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; + backup_create_params.native_copy = backup_settings.native_copy; BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. @@ -647,6 +648,7 @@ void BackupsWorker::doRestore( backup_open_params.backup_info = backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.password = restore_settings.password; + backup_open_params.native_copy = restore_settings.native_copy; BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index d12da704b2d..4dd75911a91 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -161,6 +161,7 @@ namespace M(RestoreAccessCreationMode, create_access) \ M(Bool, allow_unresolved_access_dependencies) \ M(RestoreUDFCreationMode, create_function) \ + M(Bool, native_copy) \ M(Bool, internal) \ M(String, host_id) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 3bce8698620..59d73c83d69 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -107,6 +107,9 @@ struct RestoreSettings /// How the RESTORE command will handle if a user-defined function which it's going to restore already exists. RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; + /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) + bool native_copy = true; + /// Internal, should not be specified by user. bool internal = false; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 8387b4627d5..ef8ced94590 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -107,12 +107,12 @@ void registerBackupEngineS3(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.context); + auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); return std::make_unique( backup_name_for_logging, archive_params, diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index deee71bd812..af04024d528 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -2,6 +2,14 @@ + + s3 + s3_common_disk/ + http://localhost:11111/test/common/ + clickhouse + clickhouse + 20000 + s3 s3_disk/ diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference new file mode 100644 index 00000000000..659df5e9b25 --- /dev/null +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -0,0 +1,4 @@ +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh new file mode 100755 index 00000000000..966d7ae9ce8 --- /dev/null +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag: no-fasttest - requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data; + create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_common_disk'; + insert into data select * from numbers(10); +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" From 5835e72fd6d5dd0225a0dda2f81887d6f61015fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 16:20:27 +0200 Subject: [PATCH 137/473] More generic approach to disable native copy Previous patch implements this only for BACKUP/RESTORE, but it can be useful for regular disks as well, so add allow_native_copy for disks. Note, that there is s3_allow_native_copy query setting, since it looks redundant, since it make sense only for S3 disks, and not on a per query basis. Signed-off-by: Azat Khuzhin --- src/Backups/BackupIO_S3.cpp | 34 +++++++++++++------ src/Backups/BackupIO_S3.h | 6 ++-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 14 ++++++-- src/IO/S3/copyS3File.cpp | 22 +++++++++++- src/IO/S3/copyS3File.h | 20 +++++++++-- src/Storages/StorageS3Settings.cpp | 2 ++ src/Storages/StorageS3Settings.h | 1 + 7 files changed, 80 insertions(+), 19 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 01e6bc78949..6531948c872 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,15 +101,16 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} - , native_copy(native_copy_) { + request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.allow_native_copy = native_copy; } BackupReaderS3::~BackupReaderS3() = default; @@ -139,11 +140,10 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (native_copy && destination_data_source_description.sameKind(data_source_description) + if (destination_data_source_description.sameKind(data_source_description) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { - /// Use native copy, the more optimal way. - LOG_TRACE(log, "Copying {} from S3 to disk {} using native copy", path_in_backup, destination_disk->getName()); + LOG_TRACE(log, "Copying {} from S3 to disk {}", path_in_backup, destination_disk->getName()); auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional & object_attributes) -> size_t { /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. @@ -152,7 +152,13 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); + auto create_read_buffer = [this, path_in_backup] + { + return readFile(path_in_backup); + }; + copyS3File( + create_read_buffer, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -178,16 +184,16 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} - , native_copy(native_copy_) { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.allow_native_copy = native_copy; } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -196,15 +202,23 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (native_copy && source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - /// Use native copy, the more optimal way. - LOG_TRACE(log, "Copying file {} from disk {} to S3 using native copy", src_path, src_disk->getName()); + auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] + { + if (copy_encrypted) + return src_disk->readEncryptedFile(src_path, settings); + else + return src_disk->readFile(src_path, settings); + }; + + LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( + create_read_buffer, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index d02e45370f9..16b2abfea3d 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -32,14 +32,13 @@ private: const std::shared_ptr client; S3Settings::RequestSettings request_settings; const DataSourceDescription data_source_description; - const bool native_copy; }; class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; @@ -62,7 +61,6 @@ private: S3Settings::RequestSettings request_settings; std::optional supports_batch_delete; const DataSourceDescription data_source_description; - const bool native_copy; }; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e46ca3d0828..3c19af188dc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -435,7 +435,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + auto create_read_buffer = [this, object_from] + { + return readObject(object_from); + }; + + copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -451,7 +456,12 @@ void S3ObjectStorage::copyObject( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + auto create_read_buffer = [this, object_from] + { + return readObject(object_from); + }; + + copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 7886b84cd00..3f18d3b2145 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -809,7 +809,7 @@ void copyDataToS3File( } -void copyS3File( +void copyS3FileNative( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -826,6 +826,26 @@ void copyS3File( helper.performCopy(); } +void copyS3File( + const CreateReadBuffer & create_read_buffer, + const std::shared_ptr & s3_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata, + ThreadPoolCallbackRunner schedule, + bool for_disk_s3) +{ + if (settings.allow_native_copy) + copyS3FileNative(s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + else + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); +} + } #endif diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 618ef419a9b..d41f34c103c 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -19,9 +19,9 @@ using CreateReadBuffer = std::function()>; /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyS3File() is faster and spends less network traffic and memory. +/// however copyS3FileNative() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyS3File( +void copyS3FileNative( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -51,6 +51,22 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); +/// Tries to copy file using native copy (copyS3FileNative()), if this is not +/// possible it will fallback to read-write copy (copyDataToS3File()) +void copyS3File( + const CreateReadBuffer & create_read_buffer, + const std::shared_ptr & s3_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata = std::nullopt, + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_s3 = false); + } #endif diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 89e6ee46b4d..0dc8d8d897b 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -182,6 +182,7 @@ S3Settings::RequestSettings::RequestSettings(const NamedCollection & collection) max_single_read_retries = collection.getOrDefault("max_single_read_retries", max_single_read_retries); max_connections = collection.getOrDefault("max_connections", max_connections); list_object_keys_size = collection.getOrDefault("list_object_keys_size", list_object_keys_size); + allow_native_copy = collection.getOrDefault("allow_native_copy", allow_native_copy); throw_on_zero_files_match = collection.getOrDefault("throw_on_zero_files_match", throw_on_zero_files_match); } @@ -197,6 +198,7 @@ S3Settings::RequestSettings::RequestSettings( max_connections = config.getUInt64(key + "max_connections", settings.s3_max_connections); check_objects_after_upload = config.getBool(key + "check_objects_after_upload", settings.s3_check_objects_after_upload); list_object_keys_size = config.getUInt64(key + "list_object_keys_size", settings.s3_list_object_keys_size); + allow_native_copy = config.getBool(key + "allow_native_copy", allow_native_copy); throw_on_zero_files_match = config.getBool(key + "throw_on_zero_files_match", settings.s3_throw_on_zero_files_match); retry_attempts = config.getUInt64(key + "retry_attempts", settings.s3_retry_attempts); request_timeout_ms = config.getUInt64(key + "request_timeout_ms", settings.s3_request_timeout_ms); diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 991e323acb6..581665a7dc5 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -71,6 +71,7 @@ struct S3Settings size_t retry_attempts = 10; size_t request_timeout_ms = 3000; size_t long_request_timeout_ms = 30000; // TODO: Take this from config like request_timeout_ms + bool allow_native_copy = true; bool throw_on_zero_files_match = false; From 5d63b8be0d317af7b2ee1fdfd7dc76daeeec3afd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 16:48:30 +0200 Subject: [PATCH 138/473] Add a test for allow_native_copy using clickhouse-disks (first ever) Signed-off-by: Azat Khuzhin --- .../02802_clickhouse_disks_s3_copy.reference | 4 +++ .../02802_clickhouse_disks_s3_copy.sh | 26 +++++++++++++++++++ .../02802_clickhouse_disks_s3_copy.xml | 21 +++++++++++++++ 3 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference create mode 100755 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh create mode 100644 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference new file mode 100644 index 00000000000..96860a2f90a --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference @@ -0,0 +1,4 @@ +s3_plain_native_copy +Single operation copy has completed. +s3_plain_no_native_copy +Single part upload has completed. diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh new file mode 100755 index 00000000000..f879b7a5621 --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +config="${BASH_SOURCE[0]/.sh/.xml}" + +function run_test_for_disk() +{ + local disk=$1 && shift + + echo "$disk" + + clickhouse-disks -C "$config" --disk "$disk" write --input "$config" $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --log-level test --disk "$disk" copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy |& { + grep -o -e "Single part upload has completed." -e "Single operation copy has completed." + } + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy +} + +run_test_for_disk s3_plain_native_copy +run_test_for_disk s3_plain_no_native_copy diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml new file mode 100644 index 00000000000..d4235a70903 --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml @@ -0,0 +1,21 @@ + + + + + s3_plain + http://localhost:11111/test/clickhouse-disks/ + clickhouse + clickhouse + true + + + + s3_plain + http://localhost:11111/test/clickhouse-disks/ + clickhouse + clickhouse + false + + + + From 84c720b33e9ffe44c79658af57f5985b38b8a728 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:52:00 +0200 Subject: [PATCH 139/473] Beatify test_backup_restore_s3 (using per-query profile events) Signed-off-by: Azat Khuzhin --- .../test_backup_restore_s3/test.py | 148 +++++++++--------- 1 file changed, 75 insertions(+), 73 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 0285500d044..bb14fa4824b 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -2,6 +2,7 @@ from typing import Dict, Iterable import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +import uuid cluster = ClickHouseCluster(__file__) @@ -37,32 +38,31 @@ def new_backup_name(): return f"backup{backup_id_counter}" -def get_events(events_names: Iterable[str]) -> Dict[str, int]: - _events = TSV( +def get_events_for_query(query_id: str) -> Dict[str, int]: + events = TSV( node.query( - f"SELECT event, value FROM system.events WHERE event in {events_names} SETTINGS system_events_show_zero_values = 1;" + f""" + SYSTEM FLUSH LOGS; + + WITH arrayJoin(ProfileEvents) as pe + SELECT pe.1, pe.2 + FROM system.query_log + WHERE query_id = '{query_id}' + """ ) ) return { event: int(value) - for event, value in [line.split("\t") for line in _events.lines] + for event, value in [line.split("\t") for line in events.lines] } def check_backup_and_restore( - storage_policy, backup_destination, size=1000, backup_name=None, check_events=False + storage_policy, + backup_destination, + size=1000, + backup_name=None, ): - s3_backup_events = ( - "WriteBufferFromS3Microseconds", - "WriteBufferFromS3Bytes", - "WriteBufferFromS3RequestsErrors", - ) - s3_restore_events = ( - "ReadBufferFromS3Microseconds", - "ReadBufferFromS3Bytes", - "ReadBufferFromS3RequestsErrors", - ) - node.query( f""" DROP TABLE IF EXISTS data SYNC; @@ -72,16 +72,17 @@ def check_backup_and_restore( """ ) try: - events_before_backups = get_events(s3_backup_events) - node.query(f"BACKUP TABLE data TO {backup_destination}") - events_after_backups = get_events(s3_backup_events) - events_before_restore = get_events(s3_restore_events) + backup_query_id = uuid.uuid4().hex + node.query( + f"BACKUP TABLE data TO {backup_destination}", query_id=backup_query_id + ) + restore_query_id = uuid.uuid4().hex node.query( f""" RESTORE TABLE data AS data_restored FROM {backup_destination}; - """ + """, + query_id=restore_query_id, ) - events_after_restore = get_events(s3_restore_events) node.query( """ SELECT throwIf( @@ -91,55 +92,10 @@ def check_backup_and_restore( ); """ ) - if check_events and backup_name: - objects = node.cluster.minio_client.list_objects( - "root", f"data/backups/multipart/{backup_name}/" - ) - backup_meta_size = 0 - for obj in objects: - if ".backup" in obj.object_name: - backup_meta_size = obj.size - break - backup_total_size = int( - node.query( - f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" - ).strip() - ) - restore_total_size = int( - node.query( - f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" - ).strip() - ) - # backup - # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test - assert ( - abs( - backup_total_size - - ( - events_after_backups["WriteBufferFromS3Bytes"] - - events_before_backups["WriteBufferFromS3Bytes"] - - backup_meta_size - ) - ) - < 100 - ) - assert ( - events_after_backups["WriteBufferFromS3Microseconds"] - > events_before_backups["WriteBufferFromS3Microseconds"] - ) - assert events_after_backups["WriteBufferFromS3RequestsErrors"] == 0 - # restore - assert ( - events_after_restore["ReadBufferFromS3Bytes"] - - events_before_restore["ReadBufferFromS3Bytes"] - - backup_meta_size - == restore_total_size - ) - assert ( - events_after_restore["ReadBufferFromS3Microseconds"] - > events_before_restore["ReadBufferFromS3Microseconds"] - ) - assert events_after_restore["ReadBufferFromS3RequestsErrors"] == 0 + return [ + get_events_for_query(backup_query_id), + get_events_for_query(restore_query_id), + ] finally: node.query( """ @@ -224,17 +180,63 @@ def test_backup_to_s3_multipart(): storage_policy = "default" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore( + (backup_events, restore_events) = check_backup_and_restore( storage_policy, backup_destination, size=1000000, backup_name=backup_name, - check_events=True, ) assert node.contains_in_log( f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" ) + s3_backup_events = ( + "WriteBufferFromS3Microseconds", + "WriteBufferFromS3Bytes", + "WriteBufferFromS3RequestsErrors", + ) + s3_restore_events = ( + "ReadBufferFromS3Microseconds", + "ReadBufferFromS3Bytes", + "ReadBufferFromS3RequestsErrors", + ) + + objects = node.cluster.minio_client.list_objects( + "root", f"data/backups/multipart/{backup_name}/" + ) + backup_meta_size = 0 + for obj in objects: + if ".backup" in obj.object_name: + backup_meta_size = obj.size + break + backup_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" + ).strip() + ) + restore_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" + ).strip() + ) + # backup + # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test + assert ( + abs( + backup_total_size + - (backup_events["WriteBufferFromS3Bytes"] - backup_meta_size) + ) + < 100 + ) + assert backup_events["WriteBufferFromS3Microseconds"] > 0 + assert "WriteBufferFromS3RequestsErrors" not in backup_events + # restore + assert ( + restore_events["ReadBufferFromS3Bytes"] - backup_meta_size == restore_total_size + ) + assert restore_events["ReadBufferFromS3Microseconds"] > 0 + assert "ReadBufferFromS3RequestsErrors" not in restore_events + def test_backup_to_s3_native_copy(): storage_policy = "policy_s3" From 29dc9abfcab495f66689826fdbb8ee7a81ab4c7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:58:23 +0200 Subject: [PATCH 140/473] Fix test_backup_restore_s3 after logging for native copying changed Check profile events instead of some odd logs. Signed-off-by: Azat Khuzhin --- .../test_backup_restore_s3/test.py | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index bb14fa4824b..8701bf0d832 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -244,9 +244,12 @@ def test_backup_to_s3_native_copy(): backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) - check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination + ) + # single part upload + assert backup_events["S3CopyObject"] > 0 + assert restore_events["S3CopyObject"] > 0 assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -258,9 +261,12 @@ def test_backup_to_s3_native_copy_other_bucket(): backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) - check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination + ) + # single part upload + assert backup_events["S3CopyObject"] > 0 + assert restore_events["S3CopyObject"] > 0 assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -270,9 +276,12 @@ def test_backup_to_s3_native_copy_multipart(): storage_policy = "policy_s3" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore(storage_policy, backup_destination, size=1000000) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination, size=1000000 + ) + # multi part upload + assert backup_events["S3CreateMultipartUpload"] > 0 + assert restore_events["S3CreateMultipartUpload"] > 0 assert node.contains_in_log( f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" ) From 1590ffa3b1eee26d66ae3aec3ac32c63acdea153 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 17:22:57 +0200 Subject: [PATCH 141/473] Remove copyS3FileNative() Signed-off-by: Azat Khuzhin --- src/IO/S3/copyS3File.cpp | 22 ++++------------------ src/IO/S3/copyS3File.h | 25 +++++++------------------ 2 files changed, 11 insertions(+), 36 deletions(-) diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 3f18d3b2145..2c6557d97e7 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -809,23 +809,6 @@ void copyDataToS3File( } -void copyS3FileNative( - const std::shared_ptr & s3_client, - const String & src_bucket, - const String & src_key, - size_t src_offset, - size_t src_size, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata, - ThreadPoolCallbackRunner schedule, - bool for_disk_s3) -{ - CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; - helper.performCopy(); -} - void copyS3File( const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, @@ -841,7 +824,10 @@ void copyS3File( bool for_disk_s3) { if (settings.allow_native_copy) - copyS3FileNative(s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + { + CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + helper.performCopy(); + } else copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index d41f34c103c..2c848076e9b 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -19,9 +19,14 @@ using CreateReadBuffer = std::function()>; /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyS3FileNative() is faster and spends less network traffic and memory. +/// however copyS3File() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyS3FileNative( +/// +/// Note, that it tries to copy file using native copy (CopyObject), but if it +/// has been disabled (with settings.allow_native_copy) it is fallbacks to +/// read-write copy (copyDataToS3File()). +void copyS3File( + const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -51,22 +56,6 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); -/// Tries to copy file using native copy (copyS3FileNative()), if this is not -/// possible it will fallback to read-write copy (copyDataToS3File()) -void copyS3File( - const CreateReadBuffer & create_read_buffer, - const std::shared_ptr & s3_client, - const String & src_bucket, - const String & src_key, - size_t src_offset, - size_t src_size, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata = std::nullopt, - ThreadPoolCallbackRunner schedule_ = {}, - bool for_disk_s3 = false); - } #endif From 559d3281782c22fa380e85e188d2a15e404a4c19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 17:16:02 +0200 Subject: [PATCH 142/473] Rename BACKUP setting native_copy to allow_s3_native_copy Signed-off-by: Azat Khuzhin --- src/Backups/BackupFactory.h | 2 +- src/Backups/BackupIO_S3.cpp | 8 ++++---- src/Backups/BackupIO_S3.h | 4 ++-- src/Backups/BackupSettings.cpp | 2 +- src/Backups/BackupSettings.h | 2 +- src/Backups/BackupsWorker.cpp | 4 ++-- src/Backups/RestoreSettings.cpp | 2 +- src/Backups/RestoreSettings.h | 2 +- src/Backups/registerBackupEngineS3.cpp | 4 ++-- .../0_stateless/02801_backup_native_copy.reference | 8 ++++---- tests/queries/0_stateless/02801_backup_native_copy.sh | 8 ++++---- 11 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 642f5cb07b9..e95aeddb086 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -35,7 +35,7 @@ public: std::shared_ptr backup_coordination; std::optional backup_uuid; bool deduplicate_files = true; - bool native_copy = true; + bool allow_s3_native_copy = true; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 6531948c872..60fea9e2008 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,7 +101,7 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -110,7 +110,7 @@ BackupReaderS3::BackupReaderS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = native_copy; + request_settings.allow_native_copy = allow_s3_native_copy; } BackupReaderS3::~BackupReaderS3() = default; @@ -184,7 +184,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -193,7 +193,7 @@ BackupWriterS3::BackupWriterS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = native_copy; + request_settings.allow_native_copy = allow_s3_native_copy; } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 16b2abfea3d..a93d6119786 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -38,7 +38,7 @@ private: class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 8e9fe7956f9..b6d776d0347 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes M(Bool, async) \ M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ - M(Bool, native_copy) \ + M(Bool, allow_s3_native_copy) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index e21b70ee25f..7cec2d9693d 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -39,7 +39,7 @@ struct BackupSettings bool deduplicate_files = true; /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) - bool native_copy = true; + bool allow_s3_native_copy = true; /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index fddd4f34bb6..c08b110075e 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -348,7 +348,7 @@ void BackupsWorker::doBackup( backup_create_params.backup_coordination = backup_coordination; backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; - backup_create_params.native_copy = backup_settings.native_copy; + backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. @@ -648,7 +648,7 @@ void BackupsWorker::doRestore( backup_open_params.backup_info = backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.password = restore_settings.password; - backup_open_params.native_copy = restore_settings.native_copy; + backup_open_params.allow_s3_native_copy = restore_settings.allow_s3_native_copy; BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 4dd75911a91..2009ca4c1ff 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -161,7 +161,7 @@ namespace M(RestoreAccessCreationMode, create_access) \ M(Bool, allow_unresolved_access_dependencies) \ M(RestoreUDFCreationMode, create_function) \ - M(Bool, native_copy) \ + M(Bool, allow_s3_native_copy) \ M(Bool, internal) \ M(String, host_id) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 59d73c83d69..1861e219dba 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -108,7 +108,7 @@ struct RestoreSettings RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) - bool native_copy = true; + bool allow_s3_native_copy = true; /// Internal, should not be specified by user. bool internal = false; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index ef8ced94590..bd705e4d70f 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -107,12 +107,12 @@ void registerBackupEngineS3(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); + auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); return std::make_unique( backup_name_for_logging, archive_params, diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference index 659df5e9b25..f9b008cde2e 100644 --- a/tests/queries/0_stateless/02801_backup_native_copy.reference +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -1,4 +1,4 @@ -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 -RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 -RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh index 966d7ae9ce8..015dcb19b82 100755 --- a/tests/queries/0_stateless/02801_backup_native_copy.sh +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -15,28 +15,28 @@ $CLICKHOUSE_CLIENT -nm -q " " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' From 1844ac37d76ac1a660681acb6b79af8af860d5ff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 19:12:52 +0200 Subject: [PATCH 143/473] Remove create_read_buffer argument for copyS3File() Signed-off-by: Azat Khuzhin --- src/Backups/BackupIO_S3.cpp | 15 --------------- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 14 ++------------ src/IO/S3/copyS3File.cpp | 7 ++++++- src/IO/S3/copyS3File.h | 6 +++--- 4 files changed, 11 insertions(+), 31 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 60fea9e2008..d487ec6e80e 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -152,13 +152,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); - auto create_read_buffer = [this, path_in_backup] - { - return readFile(path_in_backup); - }; - copyS3File( - create_read_buffer, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -208,17 +202,8 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] - { - if (copy_encrypted) - return src_disk->readEncryptedFile(src_path, settings); - else - return src_disk->readFile(src_path, settings); - }; - LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( - create_read_buffer, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 3c19af188dc..e46ca3d0828 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -435,12 +435,7 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - auto create_read_buffer = [this, object_from] - { - return readObject(object_from); - }; - - copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -456,12 +451,7 @@ void S3ObjectStorage::copyObject( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - auto create_read_buffer = [this, object_from] - { - return readObject(object_from); - }; - - copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 2c6557d97e7..2de2ccd0f9f 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -810,7 +810,6 @@ void copyDataToS3File( void copyS3File( - const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -829,7 +828,13 @@ void copyS3File( helper.performCopy(); } else + { + auto create_read_buffer = [&] + { + return std::make_unique(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings()); + }; copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + } } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 2c848076e9b..5d35e5ebe2d 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -23,10 +23,10 @@ using CreateReadBuffer = std::function()>; /// The parameters `src_offset` and `src_size` specify a part in the source to copy. /// /// Note, that it tries to copy file using native copy (CopyObject), but if it -/// has been disabled (with settings.allow_native_copy) it is fallbacks to -/// read-write copy (copyDataToS3File()). +/// has been disabled (with settings.allow_native_copy) or request failed +/// because it is a known issue, it is fallbacks to read-write copy +/// (copyDataToS3File()). void copyS3File( - const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, From b95836363085160a20bddfceaaf0709a0e721870 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 8 Jul 2023 14:32:34 +0200 Subject: [PATCH 144/473] tests: temporary fix the 02802_clickhouse_disks_s3_copy In #51135 the behavior of the `clickhouse-disks copy` had been changed, let's temporary update the test (and continue discussion about this change in that PR). Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh index f879b7a5621..33321607728 100755 --- a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -19,6 +19,8 @@ function run_test_for_disk() grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + # NOTE: this is due to "copy" does works like "cp -R from to/" instead of "cp from to" + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy/test clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy } From ac972661f9718b9d15e5bb49c63b2dff7d296fe3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Jul 2023 08:21:18 +0200 Subject: [PATCH 145/473] Add exclusion for "API mode: {}" from S3 in 00002_log_and_exception_messages_formatting Signed-off-by: Azat Khuzhin --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index acb6117f937..86fe01dc0e3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,7 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From 3c18a181c997f1f43e759d72eeadcc5d4f35142d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Jul 2023 13:54:18 +0200 Subject: [PATCH 146/473] Fix using of pools from the main thread Otherwise it is not possible to use clickhouse-disks with S3: $ clickhouse-disks -C /src/ch/clickhouse/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml --log-level test --disk s3_plain_native_copy copy default/test default/test.copy Failed to make request to: http://localhost:11111/test?list-type=2&max-keys=1&prefix=clickhouse-disks%2Fdefault%2Ftest.copy: Code: 49. DB::Exception: current_thread is not initialized. (LOGICAL_ERROR), Stack trace (when copying this message, always include the lines below): 0. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/exception:134: Poco::Exception::Exception(String const&, int) @ 0x000000001ad7c872 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 1. ./.cmake-llvm16/./src/Common/Exception.cpp:94: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x0000000011e2c4b7 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 2. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::Exception::Exception(int, char const (&) [34]) @ 0x000000000d341e58 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 3. ./.cmake-llvm16/./src/Common/MemoryTrackerSwitcher.h:19: DB::(anonymous namespace)::SingleEndpointHTTPSessionPool::allocObject() @ 0x0000000012010e5a in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 4. ./.cmake-llvm16/./src/Common/PoolBase.h:174: PoolBase::get(long) @ 0x0000000012011a6f in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 5. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/variant:797: DB::makePooledHTTPSession(Poco::URI const&, Poco::URI const&, DB::ConnectionTimeouts const&, unsigned long, bool, bool) @ 0x000000001200ec69 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 6. ./.cmake-llvm16/./src/IO/HTTPCommon.cpp:0: DB::makePooledHTTPSession(Poco::URI const&, DB::ConnectionTimeouts const&, unsigned long, bool, bool) @ 0x000000001200d909 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 7. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/variant:797: void DB::S3::PocoHTTPClient::makeRequestInternalImpl(Aws::Http::HttpRequest&, DB::S3::ClientConfigurationPerRequest const&, std::shared_ptr&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f5157 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 8. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::S3::PocoHTTPClient::makeRequestInternal(Aws::Http::HttpRequest&, std::shared_ptr&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f465d in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 9. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:622: DB::S3::PocoHTTPClient::MakeRequest(std::shared_ptr const&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f4454 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 10. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/source/client/AWSClient.cpp:506: Aws::Client::AWSClient::AttemptOneRequest(std::shared_ptr const&, Aws::AmazonWebServiceRequest const&, char const*, char const*, char const*) const @ 0x000000001ae2a922 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 11. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:612: Aws::Client::AWSClient::AttemptExhaustively(Aws::Http::URI const&, Aws::AmazonWebServiceRequest const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae28299 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 12. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:160: Aws::Client::AWSXMLClient::MakeRequest(Aws::Http::URI const&, Aws::AmazonWebServiceRequest const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae3c9ed in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 13. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/source/client/AWSXmlClient.cpp:66: Aws::Client::AWSXMLClient::MakeRequest(Aws::AmazonWebServiceRequest const&, Aws::Endpoint::AWSEndpoint const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae3c995 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 14. ./.cmake-llvm16/./contrib/aws/generated/src/aws-cpp-sdk-s3/source/S3Client.cpp:0: Aws::S3::S3Client::ListObjectsV2(Aws::S3::Model::ListObjectsV2Request const&) const @ 0x000000001aee6666 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 15. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:160: DB::S3::Client::ListObjectsV2(DB::S3::ExtendedRequest const&) const @ 0x00000000163cee42 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 16. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:120: DB::S3ObjectStorage::listObjects(String const&, std::vector>&, int) const @ 0x0000000016b582e2 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 17. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/vector:543: DB::IObjectStorage::existsOrHasAnyChild(String const&) const @ 0x000000001644ebe9 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 18. ./.cmake-llvm16/./src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp:0: DB::MetadataStorageFromPlainObjectStorage::exists(String const&) const @ 0x0000000016b54a64 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 19. ./.cmake-llvm16/./src/Disks/IDisk.cpp:145: DB::IDisk::copyDirectoryContent(String const&, std::shared_ptr const&, String const&) @ 0x0000000016b38fa0 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 20. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::CommandCopy::execute(std::vector> const&, std::shared_ptr&, Poco::Util::LayeredConfiguration&) @ 0x0000000012050403 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 21. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/vector:434: DB::DisksApp::main(std::vector> const&) @ 0x000000001204bf02 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 22. ./.cmake-llvm16/./base/poco/Util/src/Application.cpp:0: Poco::Util::Application::run() @ 0x000000001ac7a666 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 23. ./.cmake-llvm16/./programs/disks/DisksApp.cpp:0: mainEntryClickHouseDisks(int, char**) @ 0x000000001204c550 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 24. ./.cmake-llvm16/./programs/main.cpp:0: main @ 0x000000000cfbadc4 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 25. ? @ 0x00007ffff7dc9850 in ? 26. __libc_start_main @ 0x00007ffff7dc990a in ? 27. _start @ 0x000000000cfba1ee in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse (version 23.7.1.1) AWSXmlClient: HTTP response code: -1 Resolved remote host IP address: Request ID: Exception name: Error message: Code: 49. DB::Exception: current_thread is not initialized. (LOGICAL_ERROR) (version 23.7.1.1) 0 response headers: If the signature check failed. This could be because of a time skew. Attempting to adjust the signer. Request failed, now waiting 1600 ms before attempting again. Signed-off-by: Azat Khuzhin --- src/Common/MemoryTrackerSwitcher.h | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/MemoryTrackerSwitcher.h b/src/Common/MemoryTrackerSwitcher.h index 0fefcbb280a..3c99fd12353 100644 --- a/src/Common/MemoryTrackerSwitcher.h +++ b/src/Common/MemoryTrackerSwitcher.h @@ -6,17 +6,13 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - struct MemoryTrackerSwitcher { explicit MemoryTrackerSwitcher(MemoryTracker * new_tracker) { + /// current_thread is not initialized for the main thread, so simply do not switch anything if (!current_thread) - throw Exception(ErrorCodes::LOGICAL_ERROR, "current_thread is not initialized"); + return; auto * thread_tracker = CurrentThread::getMemoryTracker(); prev_untracked_memory = current_thread->untracked_memory; @@ -28,6 +24,10 @@ struct MemoryTrackerSwitcher ~MemoryTrackerSwitcher() { + /// current_thread is not initialized for the main thread, so simply do not switch anything + if (!current_thread) + return; + CurrentThread::flushUntrackedMemory(); auto * thread_tracker = CurrentThread::getMemoryTracker(); @@ -35,6 +35,7 @@ struct MemoryTrackerSwitcher thread_tracker->setParent(prev_memory_tracker_parent); } +private: MemoryTracker * prev_memory_tracker_parent = nullptr; Int64 prev_untracked_memory = 0; }; From 522b9ebf8c62cc564f9a2bcef5802d739a208318 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 26 Jan 2023 05:52:12 -0800 Subject: [PATCH 147/473] Implement KQL-style formatting for Interval --- .../operations/settings/settings-formats.md | 20 +++++++ src/Common/IntervalKind.cpp | 21 +++++++ src/Common/IntervalKind.h | 4 ++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 4 ++ src/Core/SettingsEnums.h | 2 + src/DataTypes/DataTypeInterval.cpp | 9 ++- src/DataTypes/DataTypeInterval.h | 1 + .../Serializations/SerializationInterval.cpp | 55 +++++++++++++++++++ .../Serializations/SerializationInterval.h | 20 +++++++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 11 ++++ src/Parsers/Kusto/ParserKQLTimespan.cpp | 27 +++++++++ src/Parsers/Kusto/ParserKQLTimespan.h | 14 +++++ ...02366_kql_native_interval_format.reference | 20 +++++++ .../02366_kql_native_interval_format.sql | 22 ++++++++ 16 files changed, 227 insertions(+), 5 deletions(-) create mode 100644 src/DataTypes/Serializations/SerializationInterval.cpp create mode 100644 src/DataTypes/Serializations/SerializationInterval.h create mode 100644 src/Parsers/Kusto/ParserKQLTimespan.cpp create mode 100644 src/Parsers/Kusto/ParserKQLTimespan.h create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.reference create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..7ab234399b9 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,6 +242,26 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## interval_format {#interval_format} + +Allows choosing different output formats of the text representation of interval types. + +Possible values: + +- `kql` - KQL-style output format. + + ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +- `numeric` - Numeric output format. + + ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +Default value: `numeric`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 411fc2886e7..9c653eefedb 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -10,6 +10,27 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +Int64 IntervalKind::toAvgNanoseconds() const +{ + static constexpr Int64 NANOSECONDS_PER_MICROSECOND = 1000; + static constexpr auto NANOSECONDS_PER_MILLISECOND = NANOSECONDS_PER_MICROSECOND * 1000; + static constexpr auto NANOSECONDS_PER_SECOND = NANOSECONDS_PER_MILLISECOND * 1000; + + switch (kind) + { + case IntervalKind::Millisecond: + return NANOSECONDS_PER_MILLISECOND; + case IntervalKind::Microsecond: + return NANOSECONDS_PER_MICROSECOND; + case IntervalKind::Nanosecond: + return 1; + default: + return toAvgSeconds() * NANOSECONDS_PER_SECOND; + } + + UNREACHABLE(); +} + Int32 IntervalKind::toAvgSeconds() const { switch (kind) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index b46805655b1..6893286f196 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -29,6 +29,10 @@ struct IntervalKind constexpr std::string_view toString() const { return magic_enum::enum_name(kind); } + /// Returns number of nanoseconds in one interval. + /// For `Month`, `Quarter` and `Year` the function returns an average number of nanoseconds. + Int64 toAvgNanoseconds() const; + /// Returns number of seconds in one interval. /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. Int32 toAvgSeconds() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..a2d030102ad 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,6 +906,7 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ + M(IntervalFormat, interval_format, FormatSettings::IntervalFormat::Numeric, "Textual representation of Interval. Possible values: 'kql', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 612f8689480..48528081258 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) +IMPLEMENT_SETTING_ENUM_WITH_RENAME(IntervalFormat, ErrorCodes::BAD_ARGUMENTS, + {{"kql", FormatSettings::IntervalFormat::KQL}, + {"numeric", FormatSettings::IntervalFormat::Numeric}}) + IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1c5be910ef7..94dc4f9aee5 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,6 +72,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(IntervalFormat, FormatSettings::IntervalFormat) + DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index 9faf0cec2d8..1649cb591cf 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -1,16 +1,16 @@ -#include -#include - +#include "DataTypeInterval.h" +#include "DataTypeFactory.h" +#include "Serializations/SerializationInterval.h" namespace DB { +SerializationPtr DataTypeInterval::doGetDefaultSerialization() const { return std::make_shared(kind); } bool DataTypeInterval::equals(const IDataType & rhs) const { return typeid(rhs) == typeid(*this) && kind == static_cast(rhs).kind; } - void registerDataTypeInterval(DataTypeFactory & factory) { factory.registerSimpleDataType("IntervalNanosecond", [] { return DataTypePtr(std::make_shared(IntervalKind::Nanosecond)); }); @@ -25,5 +25,4 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } - } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index 7de56c13b56..c398a54268e 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -24,6 +24,7 @@ public: explicit DataTypeInterval(IntervalKind kind_) : kind(kind_) {} + SerializationPtr doGetDefaultSerialization() const override; std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } String getSQLCompatibleName() const override { return "TEXT"; } diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp new file mode 100644 index 00000000000..cccb64ed390 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -0,0 +1,55 @@ +#include "SerializationInterval.h" + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; +} + +using ColumnInterval = DataTypeInterval::ColumnType; + +SerializationInterval::SerializationInterval(IntervalKind kind_) + : SerializationCustomSimpleText(DataTypeFactory::instance().get("Int64")->getDefaultSerialization()), kind(std::move(kind_)) +{ +} + +void SerializationInterval::serializeText( + const IColumn & column, const size_t row_num, WriteBuffer & ostr, const FormatSettings & format_settings) const +{ + const auto * interval_column = checkAndGetColumn(column); + if (!interval_column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + + if (const auto & format = format_settings.interval.format; format == FormatSettings::IntervalFormat::Numeric) + nested_serialization->serializeText(column, row_num, ostr, format_settings); + else if (format == FormatSettings::IntervalFormat::KQL) + { + const auto & value = interval_column->getData()[row_num]; + const auto ticks = kind.toAvgNanoseconds() * value / 100; + const auto interval_as_string = ParserKQLTimespan::compose(ticks); + ostr.write(interval_as_string.c_str(), interval_as_string.length()); + } + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); +} + +void SerializationInterval::deserializeText( + [[maybe_unused]] IColumn & column, + [[maybe_unused]] ReadBuffer & istr, + [[maybe_unused]] const FormatSettings & format_settings, + [[maybe_unused]] const bool whole) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); +} +} diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h new file mode 100644 index 00000000000..be5f914235f --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -0,0 +1,20 @@ +#pragma once + +#include "SerializationCustomSimpleText.h" + +#include + +namespace DB +{ +class SerializationInterval : public SerializationCustomSimpleText +{ +public: + explicit SerializationInterval(IntervalKind kind_); + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + +private: + IntervalKind kind; +}; +} diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ad991aa0335..fef15f0a56d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -86,6 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; + format_settings.interval.format = settings.interval_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 07d4a7ede4a..bc2e6aa7297 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -77,6 +77,17 @@ struct FormatSettings DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; + enum class IntervalFormat + { + KQL, + Numeric + }; + + struct + { + IntervalFormat format = IntervalFormat::Numeric; + } interval; + bool input_format_ipv4_default_on_conversion_error = false; bool input_format_ipv6_default_on_conversion_error = false; diff --git a/src/Parsers/Kusto/ParserKQLTimespan.cpp b/src/Parsers/Kusto/ParserKQLTimespan.cpp new file mode 100644 index 00000000000..d8ef4ed0227 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLTimespan.cpp @@ -0,0 +1,27 @@ +#include "ParserKQLTimespan.h" + +#include + +namespace DB +{ +std::string ParserKQLTimespan::compose(const Int64 ticks) +{ + static constexpr Int64 TICKS_PER_SECOND = 10000000; + static constexpr auto TICKS_PER_MINUTE = TICKS_PER_SECOND * 60; + static constexpr auto TICKS_PER_HOUR = TICKS_PER_MINUTE * 60; + static constexpr auto TICKS_PER_DAY = TICKS_PER_HOUR * 24; + + const auto abs_ticks = std::abs(ticks); + std::string result = ticks < 0 ? "-" : ""; + if (abs_ticks >= TICKS_PER_DAY) + result.append(std::format("{}.", abs_ticks / TICKS_PER_DAY)); + + result.append(std::format( + "{:02}:{:02}:{:02}", (abs_ticks / TICKS_PER_HOUR) % 24, (abs_ticks / TICKS_PER_MINUTE) % 60, (abs_ticks / TICKS_PER_SECOND) % 60)); + + if (const auto fractional_second = abs_ticks % TICKS_PER_SECOND) + result.append(std::format(".{:07}", fractional_second)); + + return result; +} +} diff --git a/src/Parsers/Kusto/ParserKQLTimespan.h b/src/Parsers/Kusto/ParserKQLTimespan.h new file mode 100644 index 00000000000..8251823253c --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLTimespan.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +#include + +namespace DB +{ +class ParserKQLTimespan +{ +public: + static std::string compose(Int64 ticks); +}; +} diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference new file mode 100644 index 00000000000..fa319341791 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -0,0 +1,20 @@ +default format +99 +100 +1 +42 +66 +66 +5 +1 +2 +kql format +00:00:00 +00:00:00.0000001 +00:00:00.0010000 +00:00:42 +01:06:00 +2.18:00:00 +5.00:00:00 +7.00:00:00 +14.00:00:00 diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql b/tests/queries/0_stateless/02366_kql_native_interval_format.sql new file mode 100644 index 00000000000..db987265bb7 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql @@ -0,0 +1,22 @@ +select 'default format'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); + +select 'kql format'; +set interval_format = 'kql'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); From 748c7fe04eef101988c555a064155a5878ba221c Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 27 Jan 2023 06:20:40 -0800 Subject: [PATCH 148/473] Refactor functional tests using Jinja2 --- ...02366_kql_native_interval_format.reference | 25 +++++++++++-------- .../02366_kql_native_interval_format.sql | 22 ---------------- .../02366_kql_native_interval_format.sql.j2 | 16 ++++++++++++ 3 files changed, 30 insertions(+), 33 deletions(-) delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index fa319341791..6588ca95e2a 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,14 +1,5 @@ -default format -99 -100 -1 -42 -66 -66 -5 -1 -2 -kql format +numeric +kql 00:00:00 00:00:00.0000001 00:00:00.0010000 @@ -18,3 +9,15 @@ kql format 5.00:00:00 7.00:00:00 14.00:00:00 +('00:01:12', '21.00:00:00', '00:00:00.0000002') +numeric +99 +100 +1 +42 +66 +66 +5 +1 +2 +('72', '3', '200') diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql b/tests/queries/0_stateless/02366_kql_native_interval_format.sql deleted file mode 100644 index db987265bb7..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql +++ /dev/null @@ -1,22 +0,0 @@ -select 'default format'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); - -select 'kql format'; -set interval_format = 'kql'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 new file mode 100644 index 00000000000..6f12dfd96db --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -0,0 +1,16 @@ +select value from system.settings where name = 'interval_format'; + +{% for format in ['kql', 'numeric'] -%} +select '{{ format }}'; +set interval_format = '{{ format }}'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); +select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200); +{% endfor -%} From b5cef61ef31fe094d4c756a5032ba2a25cc3b25d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 2 Feb 2023 10:17:24 -0800 Subject: [PATCH 149/473] Refactor solution to improve extensibility --- .../Serializations/SerializationInterval.cpp | 270 ++++++++++++++++-- .../Serializations/SerializationInterval.h | 44 ++- ...02366_kql_native_interval_format.reference | 4 +- 3 files changed, 285 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index cccb64ed390..0cd34a78f8e 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -1,5 +1,7 @@ #include "SerializationInterval.h" +#include "SerializationCustomSimpleText.h" + #include #include #include @@ -9,47 +11,263 @@ #include +#include +#include + namespace DB { +using ColumnInterval = DataTypeInterval::ColumnType; + namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +} } -using ColumnInterval = DataTypeInterval::ColumnType; - -SerializationInterval::SerializationInterval(IntervalKind kind_) - : SerializationCustomSimpleText(DataTypeFactory::instance().get("Int64")->getDefaultSerialization()), kind(std::move(kind_)) +namespace { -} - -void SerializationInterval::serializeText( - const IColumn & column, const size_t row_num, WriteBuffer & ostr, const FormatSettings & format_settings) const +class SerializationKQLInterval : public DB::SerializationCustomSimpleText { - const auto * interval_column = checkAndGetColumn(column); +public: + explicit SerializationKQLInterval(DB::IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } + + void serializeText(const DB::IColumn & column, size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const override; + void deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const override; + +private: + DB::IntervalKind kind; +}; + +void SerializationKQLInterval::serializeText( + const DB::IColumn & column, const size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings &) const +{ + const auto * interval_column = checkAndGetColumn(column); if (!interval_column) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + throw DB::Exception(DB::ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); - if (const auto & format = format_settings.interval.format; format == FormatSettings::IntervalFormat::Numeric) - nested_serialization->serializeText(column, row_num, ostr, format_settings); - else if (format == FormatSettings::IntervalFormat::KQL) - { - const auto & value = interval_column->getData()[row_num]; - const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = ParserKQLTimespan::compose(ticks); - ostr.write(interval_as_string.c_str(), interval_as_string.length()); - } - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + const auto & value = interval_column->getData()[row]; + const auto ticks = kind.toAvgNanoseconds() * value / 100; + const auto interval_as_string = DB::ParserKQLTimespan::compose(ticks); + ostr.write(interval_as_string.c_str(), interval_as_string.length()); } -void SerializationInterval::deserializeText( - [[maybe_unused]] IColumn & column, - [[maybe_unused]] ReadBuffer & istr, - [[maybe_unused]] const FormatSettings & format_settings, +void SerializationKQLInterval::deserializeText( + [[maybe_unused]] DB::IColumn & column, + [[maybe_unused]] DB::ReadBuffer & istr, + [[maybe_unused]] const DB::FormatSettings & settings, [[maybe_unused]] const bool whole) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); +} + +template Method> +void dispatch( + std::span> serializations, + const Method method, + const DB::FormatSettings::IntervalFormat format, + Args &&... args) +{ + const auto format_index = magic_enum::enum_index(format); + if (!format_index) + throw DB::Exception(DB::ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "No such format exists"); + + const auto & serialization = serializations[*format_index]; + if (!serialization) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + + (serialization.get()->*method)(std::forward(args)...); +} +} + +namespace DB +{ +SerializationInterval::SerializationInterval(IntervalKind kind_) +{ + serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::KQL).value()) + = std::make_unique(std::move(kind_)); + serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::Numeric).value()) + = std::make_unique>(); +} + +void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::deserializeBinary), + settings.interval.format, + field, + istr, + settings); +} + +void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::deserializeBinary), + settings.interval.format, + column, + istr, + settings); +} + +void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const +{ + dispatch( + serializations, + &ISerialization::deserializeBinaryBulk, + FormatSettings::IntervalFormat::Numeric, + column, + istr, + limit, + avg_value_size_hint); +} + +void SerializationInterval::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const +{ + dispatch(serializations, &ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, settings, state); +} + + +void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + dispatch( + serializations, + &ISerialization::deserializeBinaryBulkWithMultipleStreams, + FormatSettings::IntervalFormat::Numeric, + column, + limit, + settings, + state, + cache); +} + + +void SerializationInterval::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextCSV, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextEscaped, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextJSON, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextQuoted, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextRaw, settings.interval.format, column, istr, settings); +} + + +void SerializationInterval::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeWholeText, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::serializeBinary), + settings.interval.format, + field, + ostr, + settings); +} + +void SerializationInterval::serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast( + &ISerialization::serializeBinary), + settings.interval.format, + column, + row, + ostr, + settings); +} + +void SerializationInterval::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const +{ + dispatch(serializations, &ISerialization::serializeBinaryBulk, FormatSettings::IntervalFormat::Numeric, column, ostr, offset, limit); +} + +void SerializationInterval::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch( + serializations, &ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, column, settings, state); +} + +void SerializationInterval::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch(serializations, &ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalFormat::Numeric, settings, state); +} + +void SerializationInterval::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch( + serializations, + &ISerialization::serializeBinaryBulkWithMultipleStreams, + FormatSettings::IntervalFormat::Numeric, + column, + offset, + limit, + settings, + state); +} + +void SerializationInterval::serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeText, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextCSV, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextEscaped( + const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextEscaped, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextJSON, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextQuoted( + const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextQuoted, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextRaw, settings.interval.format, column, row, ostr, settings); } } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index be5f914235f..5cd13991471 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -1,20 +1,54 @@ #pragma once -#include "SerializationCustomSimpleText.h" +#include "ISerialization.h" +#include #include namespace DB { -class SerializationInterval : public SerializationCustomSimpleText +class SerializationInterval : public ISerialization { public: explicit SerializationInterval(IntervalKind kind_); - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; + void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + void serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextEscaped(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextQuoted(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; private: - IntervalKind kind; + std::array, magic_enum::enum_count()> serializations; }; } diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 6588ca95e2a..9d8043f2556 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -9,7 +9,7 @@ kql 5.00:00:00 7.00:00:00 14.00:00:00 -('00:01:12', '21.00:00:00', '00:00:00.0000002') +('00:01:12','21.00:00:00','00:00:00.0000002') numeric 99 100 @@ -20,4 +20,4 @@ numeric 5 1 2 -('72', '3', '200') +(72,3,200) From b673aa8e6b736b339b3644a3bb91bf6d6b2ccf2d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 27 Feb 2023 12:47:21 -0800 Subject: [PATCH 150/473] Use the dialect configuration --- .../operations/settings/settings-formats.md | 20 --------------- docs/en/operations/settings/settings.md | 20 +++++++++++++++ src/Core/Settings.h | 1 - src/Core/SettingsEnums.cpp | 4 --- src/Core/SettingsEnums.h | 2 -- src/Formats/FormatFactory.cpp | 11 +++++++- ...02366_kql_native_interval_format.reference | 25 +++++++++---------- .../02366_kql_native_interval_format.sql.j2 | 8 +++--- 8 files changed, 45 insertions(+), 46 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 7ab234399b9..1b22a6d1223 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,26 +242,6 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) -## interval_format {#interval_format} - -Allows choosing different output formats of the text representation of interval types. - -Possible values: - -- `kql` - KQL-style output format. - - ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. - -- `numeric` - Numeric output format. - - ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. - -Default value: `numeric`. - -See also: - -- [Interval](../../sql-reference/data-types/special-data-types/interval.md) - ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8b969f87a4d..4d5de1ace94 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,6 +98,26 @@ Default value: 0. ``` +## dialect {#dialect} + +Allows choosing support for different query languages. + +Possible values: + +- `clickhouse` - ClickHouse SQL. +- `kusto` - Microsoft KQL. +- `kusto_auto` - Tries ClickHouse SQL first, then Microsoft KQL. + +In mode `clickhouse`, ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +In modes `kusto` and `kusto_auto`, ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +Default value: `clickhouse`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a2d030102ad..3250bdf0750 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,7 +906,6 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ - M(IntervalFormat, interval_format, FormatSettings::IntervalFormat::Numeric, "Textual representation of Interval. Possible values: 'kql', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 48528081258..612f8689480 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,10 +79,6 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) -IMPLEMENT_SETTING_ENUM_WITH_RENAME(IntervalFormat, ErrorCodes::BAD_ARGUMENTS, - {{"kql", FormatSettings::IntervalFormat::KQL}, - {"numeric", FormatSettings::IntervalFormat::Numeric}}) - IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 94dc4f9aee5..1c5be910ef7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,8 +72,6 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) -DECLARE_SETTING_ENUM_WITH_RENAME(IntervalFormat, FormatSettings::IntervalFormat) - DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index fef15f0a56d..f26de67169a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -19,6 +19,15 @@ #include +namespace +{ +DB::FormatSettings::IntervalFormat convert(const DB::Dialect dialect) +{ + return dialect == DB::Dialect::kusto || dialect == DB::Dialect::kusto_auto ? DB::FormatSettings::IntervalFormat::KQL + : DB::FormatSettings::IntervalFormat::Numeric; +} +} + namespace DB { @@ -86,7 +95,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; - format_settings.interval.format = settings.interval_format; + format_settings.interval.format = convert(context->getSettingsRef().dialect); format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 9d8043f2556..06603a2fec6 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,16 +1,4 @@ -numeric -kql -00:00:00 -00:00:00.0000001 -00:00:00.0010000 -00:00:42 -01:06:00 -2.18:00:00 -5.00:00:00 -7.00:00:00 -14.00:00:00 -('00:01:12','21.00:00:00','00:00:00.0000002') -numeric +clickhouse 99 100 1 @@ -21,3 +9,14 @@ numeric 1 2 (72,3,200) +kusto_auto +00:00:00 +00:00:00.0000001 +00:00:00.0010000 +00:00:42 +01:06:00 +2.18:00:00 +5.00:00:00 +7.00:00:00 +14.00:00:00 +('00:01:12','21.00:00:00','00:00:00.0000002') diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 index 6f12dfd96db..a6b4535cc39 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -1,8 +1,6 @@ -select value from system.settings where name = 'interval_format'; - -{% for format in ['kql', 'numeric'] -%} -select '{{ format }}'; -set interval_format = '{{ format }}'; +{% for dialect in ['clickhouse', 'kusto_auto'] -%} +select '{{ dialect }}'; +set dialect = '{{ dialect }}'; select toIntervalNanosecond(99); select toIntervalNanosecond(100); select toIntervalMillisecond(1); From 2d2debe3ce7b1698b85a5771afad077daa6eae4b Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 26 Jun 2023 11:15:19 -0700 Subject: [PATCH 151/473] Introduce a separate setting for interval output formatting --- .../operations/settings/settings-formats.md | 20 +++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 2 + .../Serializations/SerializationInterval.cpp | 135 +++++------------- .../Serializations/SerializationInterval.h | 33 ++++- src/Formats/FormatFactory.cpp | 11 +- src/Formats/FormatSettings.h | 6 +- ...02366_kql_native_interval_format.reference | 25 ++-- .../02366_kql_native_interval_format.sql.j2 | 8 +- 10 files changed, 117 insertions(+), 128 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..53fd9f8ba5b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,6 +242,26 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## interval_output_format {#interval_output_format} + +Allows choosing different output formats of the text representation of interval types. + +Possible values: + +- `kusto` - KQL-style output format. + + ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +- `numeric` - Numeric output format. + + ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +Default value: `numeric`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..2a78fda03b5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,6 +906,7 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ + M(IntervalOutputFormat, interval_output_format, FormatSettings::IntervalOutputFormat::Numeric, "Textual representation of Interval. Possible values: 'kusto', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 612f8689480..1e2cbce9309 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) +IMPLEMENT_SETTING_ENUM(IntervalOutputFormat, ErrorCodes::BAD_ARGUMENTS, + {{"kusto", FormatSettings::IntervalOutputFormat::Kusto}, + {"numeric", FormatSettings::IntervalOutputFormat::Numeric}}) + IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1c5be910ef7..c2783447441 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,6 +72,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(IntervalOutputFormat, FormatSettings::IntervalOutputFormat) + DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 0cd34a78f8e..9ffa2b31ec3 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -1,19 +1,9 @@ #include "SerializationInterval.h" -#include "SerializationCustomSimpleText.h" - -#include -#include -#include -#include +#include #include #include -#include - -#include -#include - namespace DB { using ColumnInterval = DataTypeInterval::ColumnType; @@ -24,80 +14,39 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } -} -namespace +void SerializationKustoInterval::serializeText( + const IColumn & column, const size_t row, WriteBuffer & ostr, const FormatSettings &) const { -class SerializationKQLInterval : public DB::SerializationCustomSimpleText -{ -public: - explicit SerializationKQLInterval(DB::IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } - - void serializeText(const DB::IColumn & column, size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const override; - void deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const override; - -private: - DB::IntervalKind kind; -}; - -void SerializationKQLInterval::serializeText( - const DB::IColumn & column, const size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings &) const -{ - const auto * interval_column = checkAndGetColumn(column); + const auto * interval_column = checkAndGetColumn(column); if (!interval_column) - throw DB::Exception(DB::ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); const auto & value = interval_column->getData()[row]; const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = DB::ParserKQLTimespan::compose(ticks); + const auto interval_as_string = ParserKQLTimespan::compose(ticks); ostr.write(interval_as_string.c_str(), interval_as_string.length()); } -void SerializationKQLInterval::deserializeText( - [[maybe_unused]] DB::IColumn & column, - [[maybe_unused]] DB::ReadBuffer & istr, - [[maybe_unused]] const DB::FormatSettings & settings, +void SerializationKustoInterval::deserializeText( + [[maybe_unused]] IColumn & column, + [[maybe_unused]] ReadBuffer & istr, + [[maybe_unused]] const FormatSettings & settings, [[maybe_unused]] const bool whole) const { - throw DB::Exception( - DB::ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); } -template Method> -void dispatch( - std::span> serializations, - const Method method, - const DB::FormatSettings::IntervalFormat format, - Args &&... args) +SerializationInterval::SerializationInterval(IntervalKind interval_kind_) : interval_kind(std::move(interval_kind_)) { - const auto format_index = magic_enum::enum_index(format); - if (!format_index) - throw DB::Exception(DB::ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "No such format exists"); - - const auto & serialization = serializations[*format_index]; - if (!serialization) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); - - (serialization.get()->*method)(std::forward(args)...); -} -} - -namespace DB -{ -SerializationInterval::SerializationInterval(IntervalKind kind_) -{ - serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::KQL).value()) - = std::make_unique(std::move(kind_)); - serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::Numeric).value()) - = std::make_unique>(); } void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::deserializeBinary), - settings.interval.format, + settings.interval.output_format, field, istr, settings); @@ -106,9 +55,8 @@ void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::deserializeBinary), - settings.interval.format, + settings.interval.output_format, column, istr, settings); @@ -117,19 +65,13 @@ void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & ist void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const { dispatch( - serializations, - &ISerialization::deserializeBinaryBulk, - FormatSettings::IntervalFormat::Numeric, - column, - istr, - limit, - avg_value_size_hint); + &ISerialization::deserializeBinaryBulk, FormatSettings::IntervalOutputFormat::Numeric, column, istr, limit, avg_value_size_hint); } void SerializationInterval::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - dispatch(serializations, &ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, settings, state); + dispatch(&ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); } @@ -141,9 +83,8 @@ void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( SubstreamsCache * cache) const { dispatch( - serializations, &ISerialization::deserializeBinaryBulkWithMultipleStreams, - FormatSettings::IntervalFormat::Numeric, + FormatSettings::IntervalOutputFormat::Numeric, column, limit, settings, @@ -154,41 +95,40 @@ void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( void SerializationInterval::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextCSV, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextCSV, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextEscaped, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextEscaped, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextJSON, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextJSON, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextQuoted, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextQuoted, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextRaw, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextRaw, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeWholeText, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeWholeText, settings.interval.output_format, column, istr, settings); } void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::serializeBinary), - settings.interval.format, + settings.interval.output_format, field, ostr, settings); @@ -197,10 +137,9 @@ void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & o void SerializationInterval::serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { dispatch( - serializations, static_cast( &ISerialization::serializeBinary), - settings.interval.format, + settings.interval.output_format, column, row, ostr, @@ -209,29 +148,27 @@ void SerializationInterval::serializeBinary(const IColumn & column, size_t row, void SerializationInterval::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { - dispatch(serializations, &ISerialization::serializeBinaryBulk, FormatSettings::IntervalFormat::Numeric, column, ostr, offset, limit); + dispatch(&ISerialization::serializeBinaryBulk, FormatSettings::IntervalOutputFormat::Numeric, column, ostr, offset, limit); } void SerializationInterval::serializeBinaryBulkStatePrefix( const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - dispatch( - serializations, &ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, column, settings, state); + dispatch(&ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, column, settings, state); } void SerializationInterval::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - dispatch(serializations, &ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalFormat::Numeric, settings, state); + dispatch(&ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); } void SerializationInterval::serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { dispatch( - serializations, &ISerialization::serializeBinaryBulkWithMultipleStreams, - FormatSettings::IntervalFormat::Numeric, + FormatSettings::IntervalOutputFormat::Numeric, column, offset, limit, @@ -241,33 +178,33 @@ void SerializationInterval::serializeBinaryBulkWithMultipleStreams( void SerializationInterval::serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeText, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeText, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextCSV, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextCSV, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextEscaped( const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextEscaped, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextEscaped, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextJSON, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextJSON, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextQuoted( const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextQuoted, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextQuoted, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextRaw, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextRaw, settings.interval.output_format, column, row, ostr, settings); } } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index 5cd13991471..e1f0aac0e78 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -1,12 +1,26 @@ #pragma once #include "ISerialization.h" +#include "SerializationCustomSimpleText.h" +#include #include #include namespace DB { +class SerializationKustoInterval : public SerializationCustomSimpleText +{ +public: + explicit SerializationKustoInterval(IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } + + void serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; + +private: + IntervalKind kind; +}; + class SerializationInterval : public ISerialization { public: @@ -49,6 +63,23 @@ public: void serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; private: - std::array, magic_enum::enum_count()> serializations; + template Method> + void dispatch(const Method method, const FormatSettings::IntervalOutputFormat format, Args &&... args) const + { + const ISerialization * serialization = nullptr; + if (format == FormatSettings::IntervalOutputFormat::Kusto) + serialization = &serialization_kusto; + else if (format == FormatSettings::IntervalOutputFormat::Numeric) + serialization = &serialization_numeric; + + if (!serialization) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + + (serialization->*method)(std::forward(args)...); + } + + IntervalKind interval_kind; + SerializationKustoInterval serialization_kusto{interval_kind}; + SerializationNumber serialization_numeric; }; } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f26de67169a..7f33643366d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -19,15 +19,6 @@ #include -namespace -{ -DB::FormatSettings::IntervalFormat convert(const DB::Dialect dialect) -{ - return dialect == DB::Dialect::kusto || dialect == DB::Dialect::kusto_auto ? DB::FormatSettings::IntervalFormat::KQL - : DB::FormatSettings::IntervalFormat::Numeric; -} -} - namespace DB { @@ -95,7 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; - format_settings.interval.format = convert(context->getSettingsRef().dialect); + format_settings.interval.output_format = settings.interval_output_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index bc2e6aa7297..e870c61a476 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -77,15 +77,15 @@ struct FormatSettings DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; - enum class IntervalFormat + enum class IntervalOutputFormat { - KQL, + Kusto, Numeric }; struct { - IntervalFormat format = IntervalFormat::Numeric; + IntervalOutputFormat output_format = IntervalOutputFormat::Numeric; } interval; bool input_format_ipv4_default_on_conversion_error = false; diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 06603a2fec6..8a12c6885c4 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,15 +1,5 @@ -clickhouse -99 -100 -1 -42 -66 -66 -5 -1 -2 -(72,3,200) -kusto_auto +numeric +kusto 00:00:00 00:00:00.0000001 00:00:00.0010000 @@ -20,3 +10,14 @@ kusto_auto 7.00:00:00 14.00:00:00 ('00:01:12','21.00:00:00','00:00:00.0000002') +numeric +99 +100 +1 +42 +66 +66 +5 +1 +2 +(72,3,200) diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 index a6b4535cc39..0731687222d 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -1,6 +1,8 @@ -{% for dialect in ['clickhouse', 'kusto_auto'] -%} -select '{{ dialect }}'; -set dialect = '{{ dialect }}'; +select value from system.settings where name = 'interval_output_format'; + +{% for format in ['kusto', 'numeric'] -%} +select '{{ format }}'; +set interval_output_format = '{{ format }}'; select toIntervalNanosecond(99); select toIntervalNanosecond(100); select toIntervalMillisecond(1); From 54c7e6504fc0d01ad813cea44be5ebced5c91798 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 26 Jun 2023 11:55:23 -0700 Subject: [PATCH 152/473] Fix style issues --- src/DataTypes/Serializations/SerializationInterval.cpp | 1 - src/DataTypes/Serializations/SerializationInterval.h | 5 +++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 9ffa2b31ec3..a6d22481b88 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -12,7 +12,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } void SerializationKustoInterval::serializeText( diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index e1f0aac0e78..a4e6c204e4f 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class SerializationKustoInterval : public SerializationCustomSimpleText { public: From c2514279e4fdda66491ade958cb334f284d19fba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:31:30 +0300 Subject: [PATCH 153/473] Update settings.md --- docs/en/operations/settings/settings.md | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4d5de1ace94..8b969f87a4d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,26 +98,6 @@ Default value: 0. ``` -## dialect {#dialect} - -Allows choosing support for different query languages. - -Possible values: - -- `clickhouse` - ClickHouse SQL. -- `kusto` - Microsoft KQL. -- `kusto_auto` - Tries ClickHouse SQL first, then Microsoft KQL. - -In mode `clickhouse`, ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. - -In modes `kusto` and `kusto_auto`, ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. - -Default value: `clickhouse`. - -See also: - -- [Interval](../../sql-reference/data-types/special-data-types/interval.md) - ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). From 8b3848affed8dfbe61a805f7ceeca6689798e4cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:32:15 +0300 Subject: [PATCH 154/473] Update DataTypeInterval.cpp --- src/DataTypes/DataTypeInterval.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index 1649cb591cf..b8b7640300c 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -1,6 +1,6 @@ -#include "DataTypeInterval.h" -#include "DataTypeFactory.h" -#include "Serializations/SerializationInterval.h" +#include +#include +#include namespace DB { From ba267f05c9dd16c7d02f3f2ec249f72d11014712 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:32:42 +0300 Subject: [PATCH 155/473] Update DataTypeInterval.cpp --- src/DataTypes/DataTypeInterval.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index b8b7640300c..f8fe8bb3b4b 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -2,8 +2,10 @@ #include #include + namespace DB { + SerializationPtr DataTypeInterval::doGetDefaultSerialization() const { return std::make_shared(kind); } bool DataTypeInterval::equals(const IDataType & rhs) const @@ -25,4 +27,5 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } + } From 9a74b971846c74f566d3ad8a2cc466961a374d56 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 07:50:29 -0700 Subject: [PATCH 156/473] Improve naming --- .../Serializations/SerializationInterval.cpp | 4 ++-- .../{ParserKQLTimespan.cpp => Formatters.cpp} | 4 ++-- src/Parsers/Kusto/Formatters.h | 10 ++++++++++ src/Parsers/Kusto/ParserKQLTimespan.h | 14 -------------- 4 files changed, 14 insertions(+), 18 deletions(-) rename src/Parsers/Kusto/{ParserKQLTimespan.cpp => Formatters.cpp} (90%) create mode 100644 src/Parsers/Kusto/Formatters.h delete mode 100644 src/Parsers/Kusto/ParserKQLTimespan.h diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index a6d22481b88..59086d8aef3 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { @@ -23,7 +23,7 @@ void SerializationKustoInterval::serializeText( const auto & value = interval_column->getData()[row]; const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = ParserKQLTimespan::compose(ticks); + const auto interval_as_string = formatKQLTimespan(ticks); ostr.write(interval_as_string.c_str(), interval_as_string.length()); } diff --git a/src/Parsers/Kusto/ParserKQLTimespan.cpp b/src/Parsers/Kusto/Formatters.cpp similarity index 90% rename from src/Parsers/Kusto/ParserKQLTimespan.cpp rename to src/Parsers/Kusto/Formatters.cpp index d8ef4ed0227..f12af479445 100644 --- a/src/Parsers/Kusto/ParserKQLTimespan.cpp +++ b/src/Parsers/Kusto/Formatters.cpp @@ -1,10 +1,10 @@ -#include "ParserKQLTimespan.h" +#include "Formatters.h" #include namespace DB { -std::string ParserKQLTimespan::compose(const Int64 ticks) +std::string formatKQLTimespan(const Int64 ticks) { static constexpr Int64 TICKS_PER_SECOND = 10000000; static constexpr auto TICKS_PER_MINUTE = TICKS_PER_SECOND * 60; diff --git a/src/Parsers/Kusto/Formatters.h b/src/Parsers/Kusto/Formatters.h new file mode 100644 index 00000000000..16f52baf941 --- /dev/null +++ b/src/Parsers/Kusto/Formatters.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +#include + +namespace DB +{ +std::string formatKQLTimespan(Int64 ticks); +} diff --git a/src/Parsers/Kusto/ParserKQLTimespan.h b/src/Parsers/Kusto/ParserKQLTimespan.h deleted file mode 100644 index 8251823253c..00000000000 --- a/src/Parsers/Kusto/ParserKQLTimespan.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -#include - -namespace DB -{ -class ParserKQLTimespan -{ -public: - static std::string compose(Int64 ticks); -}; -} From bea58a2780d030e5d84dd8a6d29a36fe40ce75c2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 10 Jul 2023 17:21:37 -0400 Subject: [PATCH 157/473] Docs: added two methods for user profile --- docs/en/operations/settings/index.md | 91 +++++++++++++++++++++++++++- 1 file changed, 88 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index eb1d5db5676..022b50925c4 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -11,9 +11,9 @@ There are multiple ways to define ClickHouse settings. Settings are configured i The order of priority for defining a setting is: -1. Settings in the `users.xml` server configuration file - - - Set in the element ``. +1. Settings in user profiles. Set user profiles either by: + - SQL (recommended) + - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` 2. Session settings @@ -28,6 +28,91 @@ The order of priority for defining a setting is: View the [Settings](./settings.md) page for a description of the ClickHouse settings. +## Examples + +These examples all set the value of the `async_insert` setting to `1`, and show how to examine +the settings in a running system. + +### Using SQL to create a user profile + +```sql +CREATE +SETTINGS PROFILE log_ingest SETTINGS async_insert = 1 +``` + +```sql +CREATE USER ingester +IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' +# highlight-next-line +SETTINGS PROFILE log_ingest +``` + +#### Examine the user profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + +### Using XML to create a user profile + +```xml title=/etc/clickhouse-server/users.d/users.xml + +# highlight-start + + + 1 + + +# highlight-end + + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 +# highlight-start + log_ingest +# highlight-end + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 + 1 + 1 + + + +``` + +#### Examine the user profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## Converting a Setting to its Default Value If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: From e592c7c353fdc8ad7d0e1f23bfcf35fe52273490 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 11 Jul 2023 10:32:33 +0800 Subject: [PATCH 158/473] Fix integration tests - recover clickhouse_stay_alive_command lost by incomplete copy-paste --- tests/integration/helpers/cluster.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3b2c5ff40bb..8b9f75e2545 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3238,6 +3238,9 @@ class ClickHouseInstance: self.clickhouse_start_command = clickhouse_start_command.replace( "{main_config_file}", self.main_config_name ) + self.clickhouse_stay_alive_command = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( + clickhouse_start_command + ) self.path = p.join(self.cluster.instances_dir, name) self.docker_compose_path = p.join(self.path, "docker-compose.yml") @@ -4327,7 +4330,7 @@ class ClickHouseInstance: entrypoint_cmd = self.clickhouse_start_command if self.stay_alive: - entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace( + entrypoint_cmd = self.clickhouse_stay_alive_command.replace( "{main_config_file}", self.main_config_name ) else: From 1739bb306abe11210127fd384c5a278c3e59556d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 10 May 2023 02:26:32 +0800 Subject: [PATCH 159/473] Projection & optimize_aggregators_of_group_by_keys Fix projection with optimize_aggregators_of_group_by_keys = true --- src/Parsers/ASTProjectionSelectQuery.cpp | 7 +++++++ ...jection_optimize_aggregators_of_group_by_keys.reference | 1 + ...10_projection_optimize_aggregators_of_group_by_keys.sql | 7 +++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference create mode 100644 tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index da3d9286f0a..7b5889753fa 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -142,6 +142,13 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const } if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); + + auto settings_query = std::make_shared(); + SettingsChanges settings_changes; + settings_changes.insertSetting("optimize_aggregators_of_group_by_keys", false); + settings_query->changes = std::move(settings_changes); + settings_query->is_standalone = false; + select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); return node; } diff --git a/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference new file mode 100644 index 00000000000..738b2b9b918 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference @@ -0,0 +1 @@ +2012-10-23 diff --git a/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql new file mode 100644 index 00000000000..ef80dcd7143 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql @@ -0,0 +1,7 @@ +drop table if exists proj; + +CREATE TABLE proj(date Date, PROJECTION maxdate( SELECT max(date) GROUP BY date )) ENGINE = MergeTree ORDER BY tuple() as select toDate('2012-10-24')-number%100 from numbers(1e2); + +SELECT max(date) FROM proj PREWHERE date != '2012-10-24'; + +drop table proj; From a33b90c648e6797c9272a2a31b689fc58bd680f1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 16 Jun 2023 10:35:55 +0800 Subject: [PATCH 160/473] Projection & optimize_group_by_function_keys --- src/Parsers/ASTProjectionSelectQuery.cpp | 1 + ..._optimize_group_by_function_keys.reference | 0 ...ection_optimize_group_by_function_keys.sql | 31 +++++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference create mode 100644 tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 7b5889753fa..0cfdc3762a1 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -146,6 +146,7 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const auto settings_query = std::make_shared(); SettingsChanges settings_changes; settings_changes.insertSetting("optimize_aggregators_of_group_by_keys", false); + settings_changes.insertSetting("optimize_group_by_function_keys", false); settings_query->changes = std::move(settings_changes); settings_query->is_standalone = false; select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); diff --git a/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql new file mode 100644 index 00000000000..bc9b3151900 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql @@ -0,0 +1,31 @@ +drop table if exists proj; + +create table proj ( + bool_value UInt8, + zero_integer_value Int32, + integer_value Int32, + float_value Float32, + datetime_value DateTime, + string_value String, + projection test_projection ( + select + toStartOfDay (toDateTime (datetime_value)) as Day, + datetime_value, + float_value, + count( + distinct if(zero_integer_value = 1, string_value, NULL) + ) + group by + Day, + datetime_value, + float_value + ) + ) engine MergeTree +partition by + toDate (datetime_value) +order by + bool_value; + +insert into proj values (1, 1, 1, 1, '2012-10-24 21:30:00', 'ab'); + +drop table proj; From 2eccead2d421833857222bbe05e084512653386c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 11 Jul 2023 16:27:28 +0000 Subject: [PATCH 161/473] Fix const(Nullable) case, add test --- src/Functions/array/range.cpp | 8 ++++++-- tests/queries/0_stateless/02797_range_nullable.reference | 1 + tests/queries/0_stateless/02797_range_nullable.sql | 2 ++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index b14f2baca15..57679ccb180 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -404,8 +404,12 @@ private: { if (!col.type->isNullable()) return; - const auto & nullable_col = assert_cast(*col.column); - const auto & null_map = nullable_col.getNullMapData(); + const ColumnNullable * nullable_col = checkAndGetColumn(*col.column); + if (!nullable_col) + nullable_col = checkAndGetColumnConstData(col.column.get()); + if (!nullable_col) + return; + const auto & null_map = nullable_col->getNullMapData(); if (!memoryIsZero(null_map.data(), 0, null_map.size())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal (null) value column {} of argument of function {}", col.column->getName(), getName()); }; diff --git a/tests/queries/0_stateless/02797_range_nullable.reference b/tests/queries/0_stateless/02797_range_nullable.reference index a082a71f4ec..0e01ba9984c 100644 --- a/tests/queries/0_stateless/02797_range_nullable.reference +++ b/tests/queries/0_stateless/02797_range_nullable.reference @@ -5,3 +5,4 @@ [0] [0,2,4,6,8] [0,2,4,6,8] +[0,2,4,6,8] diff --git a/tests/queries/0_stateless/02797_range_nullable.sql b/tests/queries/0_stateless/02797_range_nullable.sql index 2b0fe69b123..ae35eb6fb57 100644 --- a/tests/queries/0_stateless/02797_range_nullable.sql +++ b/tests/queries/0_stateless/02797_range_nullable.sql @@ -5,6 +5,8 @@ select range('string', Null); SELECT range(toNullable(1)); SELECT range(0::Nullable(UInt64), 10::Nullable(UInt64), 2::Nullable(UInt64)); SELECT range(0::Nullable(Int64), 10::Nullable(Int64), 2::Nullable(Int64)); +SELECT range(materialize(0), 10::Nullable(UInt64), 2::Nullable(UInt64)); SELECT range(Null::Nullable(UInt64), 10::Nullable(UInt64), 2::Nullable(UInt64)); -- { serverError BAD_ARGUMENTS } SELECT range(0::Nullable(UInt64), Null::Nullable(UInt64), 2::Nullable(UInt64)); -- { serverError BAD_ARGUMENTS } SELECT range(0::Nullable(UInt64), 10::Nullable(UInt64), Null::Nullable(UInt64)); -- { serverError BAD_ARGUMENTS } +SELECT range(Null::Nullable(UInt8), materialize(1)); -- { serverError BAD_ARGUMENTS } From 1b403ab3e6ac52b0f56f071388fc07fae74379d6 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 30 May 2023 12:31:22 -0700 Subject: [PATCH 162/473] Fix crc32(WeakHash) issue for s390x --- src/Common/HashTable/Hash.h | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 189603fcbf5..87107fa9f82 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -57,28 +57,25 @@ inline DB::UInt64 intHash64(DB::UInt64 x) inline uint32_t s390x_crc32_u8(uint32_t crc, uint8_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint32_t s390x_crc32_u16(uint32_t crc, uint16_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + v = std::byteswap(v); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint32_t s390x_crc32_u32(uint32_t crc, uint32_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + v = std::byteswap(v); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint64_t s390x_crc32(uint64_t crc, uint64_t v) { - uint64_t _crc = crc; - uint32_t value_h, value_l; - value_h = (v >> 32) & 0xffffffff; - value_l = v & 0xffffffff; - _crc = crc32_be(static_cast(_crc), reinterpret_cast(&value_h), sizeof(uint32_t)); - _crc = crc32_be(static_cast(_crc), reinterpret_cast(&value_l), sizeof(uint32_t)); - return _crc; + v = std::byteswap(v); + return crc32c_le_vx(static_cast(crc), reinterpret_cast(&v), sizeof(uint64_t)); } #endif From 6086787957ce27de5510c964eff8f4c24a6bf088 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 11 Jul 2023 16:59:16 -0400 Subject: [PATCH 163/473] add overview --- .../settings.md | 4 +- docs/en/operations/settings/index.md | 175 +--------------- .../settings/settings-query-level.md | 188 ++++++++++++++++++ 3 files changed, 198 insertions(+), 169 deletions(-) create mode 100644 docs/en/operations/settings/settings-query-level.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bad7e388377..8bb7c5cb037 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1,11 +1,11 @@ --- slug: /en/operations/server-configuration-parameters/settings sidebar_position: 57 -sidebar_label: Server Settings +sidebar_label: Global Server Settings description: This section contains descriptions of server settings that cannot be changed at the session or query level. --- -# Server Settings +# Global Server Settings This section contains descriptions of server settings that cannot be changed at the session or query level. diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 022b50925c4..6863d7f3191 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -7,175 +7,16 @@ pagination_next: en/operations/settings/settings # Settings Overview -There are multiple ways to define ClickHouse settings. Settings are configured in layers, and each subsequent layer redefines the previous values of a setting. +There are two main groups of ClickHouse settings: -The order of priority for defining a setting is: +- Global server settings +- Query-level settings -1. Settings in user profiles. Set user profiles either by: - - SQL (recommended) - - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` +The main distinction between global server settings and query-level settings is that +global server settings must be set in configuration files while query-level settings +can be set in configuration files or with SQL queries. -2. Session settings +Read about [global server settings](/docs/en/operations/server-configuration-parameters/settings.md) to learn more about configuring your ClickHouse server at the global server level. - - Send `SET setting=value` from the ClickHouse console client in interactive mode. - Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` HTTP parameter. +Read about [query-level settings](/docs/en/operations/settings/settings-query-level.md) to learn more about configuring your ClickHouse server at the query-level. -3. Query settings - - - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. - - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). - - Define settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) clause of the SELECT query. The setting value is applied only to that query and is reset to the default or previous value after the query is executed. - -View the [Settings](./settings.md) page for a description of the ClickHouse settings. - -## Examples - -These examples all set the value of the `async_insert` setting to `1`, and show how to examine -the settings in a running system. - -### Using SQL to create a user profile - -```sql -CREATE -SETTINGS PROFILE log_ingest SETTINGS async_insert = 1 -``` - -```sql -CREATE USER ingester -IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' -# highlight-next-line -SETTINGS PROFILE log_ingest -``` - -#### Examine the user profile and assignment - -```sql -SHOW ACCESS -``` - -```response -┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ -│ CREATE USER default IDENTIFIED WITH sha256_password │ -# highlight-next-line -│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ -│ CREATE SETTINGS PROFILE default │ -# highlight-next-line -│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ -│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ -│ ... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -``` - -### Using XML to create a user profile - -```xml title=/etc/clickhouse-server/users.d/users.xml - -# highlight-start - - - 1 - - -# highlight-end - - - - 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 -# highlight-start - log_ingest -# highlight-end - - - 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 - 1 - 1 - - - -``` - -#### Examine the user profile and assignment - -```sql -SHOW ACCESS -``` - -```response -┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ -│ CREATE USER default IDENTIFIED WITH sha256_password │ -# highlight-next-line -│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ -│ CREATE SETTINGS PROFILE default │ -# highlight-next-line -│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ -│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ -│ ... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -``` - -## Converting a Setting to its Default Value - -If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: - -```sql -SET setting_name = DEFAULT -``` - -For example, the default value of `max_insert_block_size` is 1048449. Suppose you change its value to 100000: - -```sql -SET max_insert_block_size=100000; - -SELECT value FROM system.settings where name='max_insert_block_size'; -``` - -The response is: - -```response -┌─value──┐ -│ 100000 │ -└────────┘ -``` - -The following command sets its value back to 1048449: - -```sql -SET max_insert_block_size=DEFAULT; - -SELECT value FROM system.settings where name='max_insert_block_size'; -``` - -The setting is now back to its default: - -```response -┌─value───┐ -│ 1048449 │ -└─────────┘ -``` - - -## Custom Settings {#custom_settings} - -In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. - -A custom setting name must begin with one of predefined prefixes. The list of these prefixes must be declared in the [custom_settings_prefixes](../../operations/server-configuration-parameters/settings.md#custom_settings_prefixes) parameter in the server configuration file. - -```xml -custom_ -``` - -To define a custom setting use `SET` command: - -```sql -SET custom_a = 123; -``` - -To get the current value of a custom setting use `getSetting()` function: - -```sql -SELECT getSetting('custom_a'); -``` - -**See Also** - -- [Server Configuration Settings](../../operations/server-configuration-parameters/settings.md) diff --git a/docs/en/operations/settings/settings-query-level.md b/docs/en/operations/settings/settings-query-level.md new file mode 100644 index 00000000000..99b18e5e49f --- /dev/null +++ b/docs/en/operations/settings/settings-query-level.md @@ -0,0 +1,188 @@ +--- +sidebar_label: Query-level Settings +title: Query-level Settings +slug: /en/operations/settings/query-level +--- + +There are multiple ways to set ClickHouse query-level settings. Settings are configured in layers, and each subsequent layer redefines the previous values of a setting. + +The order of priority for defining a setting is: + +1. Settings in user profiles. Set user profiles either by: + + - SQL (recommended) + - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` + +2. Session settings + + - Send `SET setting=value` from the ClickHouse Cloud SQL console or + `clickhouse client` in interactive mode. Similarly, you can use ClickHouse + sessions in the HTTP protocol. To do this, you need to specify the + `session_id` HTTP parameter. + +3. Query settings + + - When starting `clickhouse client` in non-interactive mode, set the startup + parameter `--setting=value`. + - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). + - Define settings in the + [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) + clause of the SELECT query. The setting value is applied only to that query + and is reset to the default or previous value after the query is executed. + +## Examples + +These examples all set the value of the `async_insert` setting to `1`, and +show how to examine the settings in a running system. + +### Using SQL to create a settings profile + +This creates the profile `log_ingest` with the setting `async_inset = 1`: + +```sql +CREATE +SETTINGS PROFILE log_ingest SETTINGS async_insert = 1 +``` + +This creates the user `ingester` and assigns the user the settings profile `log_ingest`: + +```sql +CREATE USER ingester +IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' +# highlight-next-line +SETTINGS PROFILE log_ingest +``` + +#### Examine the settings profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + +### Using XML to create a settings profile + +```xml title=/etc/clickhouse-server/users.d/users.xml + +# highlight-start + + + 1 + + +# highlight-end + + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 +# highlight-start + log_ingest +# highlight-end + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 + 1 + 1 + + + +``` + +#### Examine the settings profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + +## Converting a Setting to its Default Value + +If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: + +```sql +SET setting_name = DEFAULT +``` + +For example, the default value of `max_insert_block_size` is 1048449. Suppose you change its value to 100000: + +```sql +SET max_insert_block_size=100000; + +SELECT value FROM system.settings where name='max_insert_block_size'; +``` + +The response is: + +```response +┌─value──┐ +│ 100000 │ +└────────┘ +``` + +The following command sets its value back to 1048449: + +```sql +SET max_insert_block_size=DEFAULT; + +SELECT value FROM system.settings where name='max_insert_block_size'; +``` + +The setting is now back to its default: + +```response +┌─value───┐ +│ 1048449 │ +└─────────┘ +``` + + +## Custom Settings {#custom_settings} + +In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. + +A custom setting name must begin with one of predefined prefixes. The list of these prefixes must be declared in the [custom_settings_prefixes](../../operations/server-configuration-parameters/settings.md#custom_settings_prefixes) parameter in the server configuration file. + +```xml +custom_ +``` + +To define a custom setting use `SET` command: + +```sql +SET custom_a = 123; +``` + +To get the current value of a custom setting use `getSetting()` function: + +```sql +SELECT getSetting('custom_a'); +``` + +**See Also** + +- View the [Settings](./settings.md) page for a description of the ClickHouse settings. +- [Global server settings](../../operations/server-configuration-parameters/settings.md) From 46fd88dca4bbcf6b83843780c4579de4212e0222 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 11 Jul 2023 17:50:52 -0400 Subject: [PATCH 164/473] add session setting --- .../settings/settings-query-level.md | 89 ++++++++++++------- 1 file changed, 59 insertions(+), 30 deletions(-) diff --git a/docs/en/operations/settings/settings-query-level.md b/docs/en/operations/settings/settings-query-level.md index 99b18e5e49f..81cc2294a4c 100644 --- a/docs/en/operations/settings/settings-query-level.md +++ b/docs/en/operations/settings/settings-query-level.md @@ -8,7 +8,7 @@ There are multiple ways to set ClickHouse query-level settings. Settings are con The order of priority for defining a setting is: -1. Settings in user profiles. Set user profiles either by: +1. Applying a setting to a user directly, or within a settings profile - SQL (recommended) - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` @@ -35,7 +35,32 @@ The order of priority for defining a setting is: These examples all set the value of the `async_insert` setting to `1`, and show how to examine the settings in a running system. -### Using SQL to create a settings profile +### Using SQL to apply a setting to a user directly + +This creates the user `ingester` with the setting `async_inset = 1`: + +```sql +CREATE USER ingester +IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' +# highlight-next-line +SETTINGS async_insert = 1 +``` + +#### Examine the settings profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ ... │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS async_insert = true │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` +### Using SQL to create a settings profile and assign to a user This creates the profile `log_ingest` with the setting `async_inset = 1`: @@ -53,26 +78,8 @@ IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f2 SETTINGS PROFILE log_ingest ``` -#### Examine the settings profile and assignment -```sql -SHOW ACCESS -``` - -```response -┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ -│ CREATE USER default IDENTIFIED WITH sha256_password │ -# highlight-next-line -│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ -│ CREATE SETTINGS PROFILE default │ -# highlight-next-line -│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ -│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ -│ ... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -``` - -### Using XML to create a settings profile +### Using XML to create a settings profile and user ```xml title=/etc/clickhouse-server/users.d/users.xml @@ -119,6 +126,29 @@ SHOW ACCESS └────────────────────────────────────────────────────────────────────────────────────┘ ``` +### Assign a setting to a session + +```sql +SET async_insert =1; +SELECT value FROM system.settings where name='async_insert'; +``` + +```response +┌─value──┐ +│ 1 │ +└────────┘ +``` + +### Assign a setting during a query + +```sql +INSERT INTO YourTable +# highlight-next-line +SETTINGS async_insert=1 +VALUES (...) +``` + + ## Converting a Setting to its Default Value If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: @@ -127,39 +157,38 @@ If you change a setting and would like to revert it back to its default value, s SET setting_name = DEFAULT ``` -For example, the default value of `max_insert_block_size` is 1048449. Suppose you change its value to 100000: +For example, the default value of `async_insert` is `0`. Suppose you change its value to `1`: ```sql -SET max_insert_block_size=100000; +SET async_insert = 1; -SELECT value FROM system.settings where name='max_insert_block_size'; +SELECT value FROM system.settings where name='async_insert'; ``` The response is: ```response ┌─value──┐ -│ 100000 │ +│ 1 │ └────────┘ ``` -The following command sets its value back to 1048449: +The following command sets its value back to 0: ```sql -SET max_insert_block_size=DEFAULT; +SET async_insert = DEFAULT; -SELECT value FROM system.settings where name='max_insert_block_size'; +SELECT value FROM system.settings where name='async_insert'; ``` The setting is now back to its default: ```response ┌─value───┐ -│ 1048449 │ +│ 0 │ └─────────┘ ``` - ## Custom Settings {#custom_settings} In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. From 52246ab5b544a3fa5c9482871f90e5a2517c2710 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 11 Jul 2023 23:23:15 +0800 Subject: [PATCH 165/473] Ignore setting constraints for projection ASTs --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/InterpreterSetQuery.cpp | 5 +- src/Interpreters/InterpreterSetQuery.h | 2 +- src/Interpreters/SelectQueryOptions.h | 8 +++ .../optimizeUseAggregateProjection.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 50 ++++++++++++------- 6 files changed, 47 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4..32812151b59 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3181,7 +3181,7 @@ void InterpreterSelectQuery::initSettings() { auto & query = getSelectQuery(); if (query.settings()) - InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + InterpreterSetQuery(query.settings(), context).executeForCurrentContext(options.ignore_setting_constraints); auto & client_info = context->getClientInfo(); auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index c25de7c55ea..6db57a4f950 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -24,10 +24,11 @@ BlockIO InterpreterSetQuery::execute() } -void InterpreterSetQuery::executeForCurrentContext() +void InterpreterSetQuery::executeForCurrentContext(bool ignore_setting_constraints) { const auto & ast = query_ptr->as(); - getContext()->checkSettingsConstraints(ast.changes); + if (!ignore_setting_constraints) + getContext()->checkSettingsConstraints(ast.changes); getContext()->applySettingsChanges(ast.changes); getContext()->resetSettingsToDefaultValue(ast.default_settings); } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index bcd4022f9bb..2438762f347 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -23,7 +23,7 @@ public: /** Set setting for current context (query context). * It is used for interpretation of SETTINGS clause in SELECT query. */ - void executeForCurrentContext(); + void executeForCurrentContext(bool ignore_setting_constraints = false); bool supportsTransactions() const override { return true; } diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index e6895ed243b..c91329c869c 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -51,6 +51,8 @@ struct SelectQueryOptions bool settings_limit_offset_done = false; bool is_explain = false; /// The value is true if it's explain statement. bool is_create_parameterized_view = false; + /// Bypass setting constraints for some internal queries such as projection ASTs. + bool ignore_setting_constraints = false; /// These two fields are used to evaluate shardNum() and shardCount() function when /// prefer_localhost_replica == 1 and local instance is selected. They are needed because local @@ -141,6 +143,12 @@ struct SelectQueryOptions return *this; } + SelectQueryOptions & ignoreSettingConstraints(bool value = true) + { + ignore_setting_constraints = value; + return *this; + } + SelectQueryOptions & setInternal(bool value = false) { is_internal = value; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 2959178b2e5..6e76d6e67a7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -69,7 +69,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( projection.query_ast, context, Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations().ignoreSettingConstraints()); const auto & analysis_result = interpreter.getAnalysisResult(); const auto & query_analyzer = interpreter.getQueryAnalyzer(); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index e568fba0495..73fb279d51c 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -7,18 +7,18 @@ #include #include #include +#include #include #include -#include #include #include -#include -#include -#include -#include #include #include +#include +#include +#include +#include #include @@ -109,9 +109,16 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, + result.query_ast, + query_context, + storage, + {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState} + .modify() + .ignoreAlias() + .ignoreASTOptimizations() + .ignoreSettingConstraints()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -220,9 +227,16 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, + result.query_ast, + query_context, + storage, + {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState} + .modify() + .ignoreAlias() + .ignoreASTOptimizations() + .ignoreSettingConstraints()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -241,7 +255,8 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.sample_block_for_keys.insert({nullptr, key.type, key.name}); auto it = partition_column_name_to_value_index.find(key.name); if (it == partition_column_name_to_value_index.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "minmax_count projection can only have keys about partition columns. It's a bug"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "minmax_count projection can only have keys about partition columns. It's a bug"); result.partition_value_indices.push_back(it->second); } } @@ -282,7 +297,8 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) Pipe(std::make_shared(block)), SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns - : QueryProcessingStage::WithMergeableState}) + : QueryProcessingStage::WithMergeableState} + .ignoreSettingConstraints()) .buildQueryPipeline(); builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. @@ -353,8 +369,8 @@ void ProjectionsDescription::add(ProjectionDescription && projection, const Stri { if (if_not_exists) return; - throw Exception(ErrorCodes::ILLEGAL_PROJECTION, "Cannot add projection {}: projection with this name already exists", - projection.name); + throw Exception( + ErrorCodes::ILLEGAL_PROJECTION, "Cannot add projection {}: projection with this name already exists", projection.name); } auto insert_it = projections.cend(); @@ -363,10 +379,10 @@ void ProjectionsDescription::add(ProjectionDescription && projection, const Stri insert_it = projections.cbegin(); else if (!after_projection.empty()) { - auto it = std::find_if(projections.cbegin(), projections.cend(), [&after_projection](const auto & projection_) - { - return projection_.name == after_projection; - }); + auto it = std::find_if( + projections.cbegin(), + projections.cend(), + [&after_projection](const auto & projection_) { return projection_.name == after_projection; }); if (it != projections.cend()) ++it; insert_it = it; From 3bd9c57a8cfc4210e421690b597afa7ba68a36d0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Jul 2023 09:29:36 +0000 Subject: [PATCH 166/473] allow to ignore CREATE INDEX query for better SQL syntax compatibility --- src/Core/Settings.h | 1 + .../InterpreterCreateIndexQuery.cpp | 17 ++++++++ src/Parsers/ASTIndexDeclaration.cpp | 41 +++++++++++-------- src/Parsers/ParserCreateIndexQuery.cpp | 29 +++++++++---- 4 files changed, 64 insertions(+), 24 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..8f61c3973e3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,6 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ + M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 714bcd6d356..0e36fa22404 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -15,6 +15,8 @@ namespace DB namespace ErrorCodes { extern const int TABLE_IS_READ_ONLY; + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; } @@ -23,6 +25,21 @@ BlockIO InterpreterCreateIndexQuery::execute() auto current_context = getContext(); const auto & create_index = query_ptr->as(); + // Noop if allow_create_index_without_type = true. throw otherwise + if (!create_index.index_decl->as()->type) + { + if (!current_context->getSettingsRef().allow_create_index_without_type) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, "CREATE INDEX without TYPE is forbidden." + " SET allow_create_index_without_type=1 to ignore this statements."); + } + else + { + // Nothing to do + return {}; + } + } + AccessRightsElements required_access; required_access.emplace_back(AccessType::ALTER_ADD_INDEX, create_index.getDatabase(), create_index.getTable()); diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index d223661451e..e647f02f553 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -13,8 +13,8 @@ ASTPtr ASTIndexDeclaration::clone() const auto res = std::make_shared(); res->name = name; - res->granularity = granularity; - + if (granularity) + res->granularity = granularity; if (expr) res->set(res->expr, expr->clone()); if (type) @@ -25,23 +25,32 @@ ASTPtr ASTIndexDeclaration::clone() const void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { - if (part_of_create_index_query) + if (expr) { - s.ostr << "("; - expr->formatImpl(s, state, frame); - s.ostr << ")"; - } - else - { - s.ostr << backQuoteIfNeed(name); - s.ostr << " "; - expr->formatImpl(s, state, frame); + if (part_of_create_index_query) + { + s.ostr << "("; + expr->formatImpl(s, state, frame); + s.ostr << ")"; + } + else + { + s.ostr << backQuoteIfNeed(name); + s.ostr << " "; + expr->formatImpl(s, state, frame); + } } - s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); - type->formatImpl(s, state, frame); - s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : ""); - s.ostr << granularity; + if (type) + { + s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); + type->formatImpl(s, state, frame); + } + if (granularity) + { + s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : ""); + s.ostr << granularity; + } } } diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index f231573b920..5bcea50af87 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -17,24 +17,36 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected { ParserKeyword s_type("TYPE"); ParserKeyword s_granularity("GRANULARITY"); - + ParserToken open(TokenType::OpeningRoundBracket); + ParserToken close(TokenType::ClosingRoundBracket); + ParserOrderByExpressionList order_list; ParserDataType data_type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; ASTPtr expr; + ASTPtr order; ASTPtr type; ASTPtr granularity; + if (open.ignore(pos, expected)) + { + if (!order_list.parse(pos, order, expected)) + return false; + + if (!close.ignore(pos, expected)) + return false; + } /// Skip name parser for SQL-standard CREATE INDEX - if (!expression_p.parse(pos, expr, expected)) + else if (!expression_p.parse(pos, expr, expected)) return false; - if (!s_type.ignore(pos, expected)) - return false; - if (!data_type_p.parse(pos, type, expected)) - return false; + if (s_type.ignore(pos, expected)) + { + if (!data_type_p.parse(pos, type, expected)) + return false; + } if (s_granularity.ignore(pos, expected)) { @@ -45,13 +57,14 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected auto index = std::make_shared(); index->part_of_create_index_query = true; index->set(index->expr, expr); - index->set(index->type, type); + if (type) + index->set(index->type, type); if (granularity) index->granularity = granularity->as().value.safeGet(); else { - if (index->type->name == "annoy") + if (index->type && index->type->name == "annoy") index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; From 31806d0b6c6726e6a2540587735ba9449c012062 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Jul 2023 11:14:49 +0000 Subject: [PATCH 167/473] no style --- src/Interpreters/InterpreterCreateIndexQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 0e36fa22404..752bc6200ce 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { extern const int TABLE_IS_READ_ONLY; - extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; } From 2ee7a9846f0b31b7a792f877ae2c8c02cb5bad5a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Jul 2023 12:47:24 +0000 Subject: [PATCH 168/473] fix --- src/Parsers/ParserCreateIndexQuery.cpp | 10 +++++----- src/Storages/IndicesDescription.cpp | 13 +++++++++++-- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 5bcea50af87..d2ae7f972b7 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -29,7 +29,11 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ASTPtr type; ASTPtr granularity; - if (open.ignore(pos, expected)) + /// Skip name parser for SQL-standard CREATE INDEX + if (expression_p.parse(pos, expr, expected)) + { + } + else if (open.ignore(pos, expected)) { if (!order_list.parse(pos, order, expected)) return false; @@ -37,10 +41,6 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (!close.ignore(pos, expected)) return false; } - /// Skip name parser for SQL-standard CREATE INDEX - else if (!expression_p.parse(pos, expr, expected)) - return false; - if (s_type.ignore(pos, expected)) { diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index a93ac248c98..c7aeaf8e4ef 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -11,6 +11,7 @@ #include #include +#include "Common/Exception.h" namespace DB @@ -89,8 +90,16 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast result.type = Poco::toLower(index_definition->type->name); result.granularity = index_definition->granularity; - ASTPtr expr_list = extractKeyExpressionList(index_definition->expr->clone()); - result.expression_list_ast = expr_list->clone(); + ASTPtr expr_list; + if (index_definition->expr) + { + expr_list = extractKeyExpressionList(index_definition->expr->clone()); + result.expression_list_ast = expr_list->clone(); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expression is not set"); + } auto syntax = TreeRewriter(context).analyze(expr_list, columns.getAllPhysical()); result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true); From d891c4c9a218784d1717710a676b3a132339b14b Mon Sep 17 00:00:00 2001 From: Lars Eidnes Date: Wed, 12 Jul 2023 15:06:01 +0200 Subject: [PATCH 169/473] MaterializedMySQL: Delay disconnection of mysqlxx::Pool::Entry until refcount==0 If Entry::disconnect() is called, it is removed from its pool and underlying mysqlxx::Connection is disconnected. Since Entry objects are copyable and some of them could be alive when disconnect() is called which would lead to undefined behavior. Suggesting to disconnect it when the last object is being destroyed. --- src/Common/mysqlxx/Pool.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index d10889d1f97..64a69c48e1d 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -25,8 +25,6 @@ void Pool::Entry::incrementRefCount() /// First reference, initialize thread if (data->ref_count.fetch_add(1) == 0) mysql_thread_init(); - - chassert(!data->removed_from_pool); } @@ -43,7 +41,10 @@ void Pool::Entry::decrementRefCount() /// In Pool::Entry::disconnect() we remove connection from the list of pool's connections. /// So now we must deallocate the memory. if (data->removed_from_pool) + { + data->conn.disconnect(); ::delete data; + } } } @@ -230,8 +231,6 @@ void Pool::removeConnection(Connection* connection) std::lock_guard lock(mutex); if (connection) { - if (!connection->removed_from_pool) - connection->conn.disconnect(); connections.remove(connection); connection->removed_from_pool = true; } @@ -240,6 +239,7 @@ void Pool::removeConnection(Connection* connection) void Pool::Entry::disconnect() { + // Remove the Entry from the Pool. Actual disconnection is delayed until refcount == 0. pool->removeConnection(data); } From 771b1f8f475993b331749af9d7edff5fc2e5866c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 12 Jul 2023 15:48:09 +0000 Subject: [PATCH 170/473] Prototype (working) --- programs/server/Server.cpp | 8 +---- src/Common/SystemLogBase.cpp | 54 ++++++++++++++++++++--------- src/Common/SystemLogBase.h | 61 ++++++++++++++++++++++++--------- src/Interpreters/SystemLog.cpp | 26 +++++++------- src/Interpreters/SystemLog.h | 17 ++++----- src/Interpreters/TextLog.cpp | 2 +- src/Interpreters/TextLog.h | 8 +++++ src/Loggers/Loggers.cpp | 18 ++++------ src/Loggers/Loggers.h | 13 ------- src/Loggers/OwnSplitChannel.cpp | 6 ++-- src/Loggers/OwnSplitChannel.h | 8 +++-- 11 files changed, 129 insertions(+), 92 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c7a7ba71e83..79a28558d93 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1599,13 +1599,7 @@ try global_context->setSystemZooKeeperLogAfterInitializationIfNeeded(); /// Build loggers before tables startup to make log messages from tables /// attach available in system.text_log - { - String level_str = config().getString("text_log.level", ""); - int level = level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(level_str); - setTextLog(global_context->getTextLog(), level); - - buildLoggers(config(), logger()); - } + buildLoggers(config(), logger()); /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) attachSystemTablesServer(global_context, *database_catalog.getSystemDatabase(), has_zookeeper); attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA)); diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 86adcbbd31b..28dc5ec2666 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -38,10 +38,20 @@ namespace ISystemLog::~ISystemLog() = default; -void ISystemLog::stopFlushThread() +template +SystemLogBase::SystemLogBase(std::shared_ptr> ex_queue) +{ + if (ex_queue) + queue = ex_queue; + else + queue = std::make_shared>(); +} + +template +void SystemLogBase::stopFlushThread() { { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); if (!saving_thread || !saving_thread->joinable()) return; @@ -52,22 +62,26 @@ void ISystemLog::stopFlushThread() is_shutdown = true; /// Tell thread to shutdown. - flush_event.notify_all(); + queue->flush_event.notify_all(); } saving_thread->join(); } -void ISystemLog::startup() +template +void SystemLogBase::startup() { - std::lock_guard lock(mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); + std::lock_guard lock(queue->mutex); + std::cout << "void ISystemLog::startup()" << std::endl; + saving_thread = std::make_unique([this] { + std::cout << "void ISystemLog::ThreadFromGlobalPool()" << std::endl; + savingThreadFunction(); }); } static thread_local bool recursive_add_call = false; template -void SystemLogBase::add(const LogElement & element) +void SystemLogQueue::add(const LogElement & element) { /// It is possible that the method will be called recursively. /// Better to drop these events to avoid complications. @@ -75,21 +89,21 @@ void SystemLogBase::add(const LogElement & element) return; recursive_add_call = true; SCOPE_EXIT({ recursive_add_call = false; }); - - /// Memory can be allocated while resizing on queue.push_back. + /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + /// Should not log messages under mutex. bool queue_is_half_full = false; { std::unique_lock lock(mutex); - if (is_shutdown) - return; + // if (queue.is_shutdown) + // return; // TODO if (queue.size() == DBMS_SYSTEM_LOG_QUEUE_SIZE / 2) { @@ -134,25 +148,31 @@ void SystemLogBase::add(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogBase::add(const LogElement & element) +{ + queue->add(element); +} + template void SystemLogBase::flush(bool force) { uint64_t this_thread_requested_offset; { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); if (is_shutdown) return; - this_thread_requested_offset = queue_front_index + queue.size(); + this_thread_requested_offset = queue->queue_front_index + queue->size(); // Publish our flush request, taking care not to overwrite the requests // made by other threads. is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + queue->requested_flush_up_to = std::max(queue->requested_flush_up_to, this_thread_requested_offset); - flush_event.notify_all(); + queue->flush_event.notify_all(); } LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); @@ -161,8 +181,8 @@ void SystemLogBase::flush(bool force) // too fast for our parallel functional tests, probably because they // heavily load the disk. const int timeout_seconds = 180; - std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + std::unique_lock lock(queue->mutex); + bool result = queue->flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { return flushed_up_to >= this_thread_requested_offset && !is_force_prepare_tables; }); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index f8febd8b159..cb6003c94ef 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -55,25 +55,52 @@ public: virtual void prepareTable() = 0; /// Start the background thread. - virtual void startup(); + virtual void startup() = 0; /// Stop the background flush thread before destructor. No more data will be written. virtual void shutdown() = 0; + virtual void stopFlushThread() = 0; + virtual ~ISystemLog(); virtual void savingThreadFunction() = 0; protected: - std::unique_ptr saving_thread; + std::unique_ptr saving_thread; +}; + +template +class SystemLogQueue +{ +public: + + void add(const LogElement & element); + size_t size() const { return queue.size(); } + void push_back(const LogElement & element) + { + queue.push_back(element); + } + + // Queue is bounded. But its size is quite large to not block in all normal cases. + std::vector queue; + // An always-incrementing index of the first message currently in the queue. + // We use it to give a global sequential index to every message, so that we + // can wait until a particular message is flushed. This is used to implement + // synchronous log flushing for SYSTEM FLUSH LOGS. + uint64_t queue_front_index = 0; /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; - - bool is_shutdown = false; std::condition_variable flush_event; - void stopFlushThread(); + // Requested to flush logs up to this index, exclusive + uint64_t requested_flush_up_to = 0; + + // Logged overflow message at this queue front index + uint64_t logged_queue_full_at_index = -1; + + Poco::Logger * log; }; template @@ -82,6 +109,8 @@ class SystemLogBase : public ISystemLog public: using Self = SystemLogBase; + SystemLogBase(std::shared_ptr> ex_queue = nullptr); + /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. */ @@ -90,6 +119,10 @@ public: /// Flush data in the buffer to disk void flush(bool force) override; + void startup() override; + + void stopFlushThread() override; + String getName() const override { return LogElement::name(); } static const char * getDefaultOrderBy() { return "event_date, event_time"; } @@ -97,21 +130,17 @@ public: protected: Poco::Logger * log; - // Queue is bounded. But its size is quite large to not block in all normal cases. - std::vector queue; - // An always-incrementing index of the first message currently in the queue. - // We use it to give a global sequential index to every message, so that we - // can wait until a particular message is flushed. This is used to implement - // synchronous log flushing for SYSTEM FLUSH LOGS. - uint64_t queue_front_index = 0; + std::shared_ptr> queue; + // A flag that says we must create the tables even if the queue is empty. bool is_force_prepare_tables = false; - // Requested to flush logs up to this index, exclusive - uint64_t requested_flush_up_to = 0; + // Flushed log up to this index, exclusive uint64_t flushed_up_to = 0; - // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; + + + bool is_shutdown = false; + }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3fd0297f5b8..cbf355d020a 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -332,8 +332,10 @@ SystemLog::SystemLog( const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_) - : WithContext(context_) + size_t flush_interval_milliseconds_, + std::shared_ptr> ex_queue) + : Base(ex_queue) + , WithContext(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) @@ -371,21 +373,21 @@ void SystemLog::savingThreadFunction() bool should_prepare_tables_anyway = false; { - std::unique_lock lock(mutex); - flush_event.wait_for(lock, + std::unique_lock lock(queue->mutex); + queue->flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds), [&] () { - return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; + return queue->requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; } ); - queue_front_index += queue.size(); - to_flush_end = queue_front_index; + queue->queue_front_index += queue->size(); + to_flush_end = queue->queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. to_flush.resize(0); - queue.swap(to_flush); + queue->queue.swap(to_flush); should_prepare_tables_anyway = is_force_prepare_tables; @@ -399,9 +401,9 @@ void SystemLog::savingThreadFunction() prepareTable(); LOG_TRACE(log, "Table created (force)"); - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); is_force_prepare_tables = false; - flush_event.notify_all(); + queue->flush_event.notify_all(); } } else @@ -474,10 +476,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, } { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); flushed_up_to = to_flush_end; is_force_prepare_tables = false; - flush_event.notify_all(); + queue->flush_event.notify_all(); } LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 84b70c67e2a..bf41ff12d2a 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -108,22 +108,23 @@ public: const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + std::shared_ptr> ex_queue = nullptr); void shutdown() override; protected: - using ISystemLog::mutex; - using ISystemLog::is_shutdown; - using ISystemLog::flush_event; - using ISystemLog::stopFlushThread; + //using ISystemLog::mutex; + using Base::is_shutdown; + // using ISystemLog::flush_event; + using Base::stopFlushThread; using Base::log; using Base::queue; - using Base::queue_front_index; + // using Base::queue_front_index; using Base::is_force_prepare_tables; - using Base::requested_flush_up_to; + //using Base::requested_flush_up_to; using Base::flushed_up_to; - using Base::logged_queue_full_at_index; + // using Base::logged_queue_full_at_index; private: diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index 45d5a7b2344..a66092c1c2b 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -84,7 +84,7 @@ TextLog::TextLog(ContextPtr context_, const String & database_name_, const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_) : SystemLog(context_, database_name_, table_name_, - storage_def_, flush_interval_milliseconds_) + storage_def_, flush_interval_milliseconds_, getLogQueue()) { // SystemLog methods may write text logs, so we disable logging for the text // log table to avoid recursion. diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 6efc1c906d4..9c761f9d865 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -46,6 +46,14 @@ public: const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_); + + static std::shared_ptr> getLogQueue() + { + static std::shared_ptr> queue = std::make_shared>(); + return queue; + } }; + + } diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 0c3a7bd615d..96793ee5e0a 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,21 +34,9 @@ static std::string createDirectory(const std::string & file) return path; } -#ifndef WITHOUT_TEXT_LOG -void Loggers::setTextLog(std::shared_ptr log, int max_priority) -{ - text_log = log; - text_log_max_priority = max_priority; -} -#endif void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { -#ifndef WITHOUT_TEXT_LOG - if (split) - if (auto log = text_log.lock()) - split->addTextLog(log, text_log_max_priority); -#endif auto current_logger = config.getString("logger", ""); if (config_logger.has_value() && *config_logger == current_logger) @@ -62,6 +50,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// Use extended interface of Channel for more comprehensive logging. split = new DB::OwnSplitChannel(); +#ifndef WITHOUT_TEXT_LOG + String text_log_level_str = config.getString("text_log.level", ""); + int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); +#endif + auto log_level_string = config.getString("logger.level", "trace"); /// different channels (log, console, syslog) may have different loglevels configured diff --git a/src/Loggers/Loggers.h b/src/Loggers/Loggers.h index ebc10954b94..9eff731a4c5 100644 --- a/src/Loggers/Loggers.h +++ b/src/Loggers/Loggers.h @@ -7,12 +7,6 @@ #include #include "OwnSplitChannel.h" -#ifndef WITHOUT_TEXT_LOG -namespace DB -{ - class TextLog; -} -#endif namespace Poco::Util { @@ -29,9 +23,6 @@ public: /// Close log files. On next log write files will be reopened. void closeLogs(Poco::Logger & logger); -#ifndef WITHOUT_TEXT_LOG - void setTextLog(std::shared_ptr log, int max_priority); -#endif private: Poco::AutoPtr log_file; @@ -41,10 +32,6 @@ private: /// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed. std::optional config_logger; -#ifndef WITHOUT_TEXT_LOG - std::weak_ptr text_log; - int text_log_max_priority = -1; -#endif Poco::AutoPtr split; }; diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 03db198c305..7661996ea33 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -135,7 +135,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) elem.source_line = msg.getSourceLine(); elem.message_format_string = msg.getFormatString(); - std::shared_ptr text_log_locked{}; + std::shared_ptr> text_log_locked{}; { std::lock_guard lock(text_log_mutex); text_log_locked = text_log.lock(); @@ -153,10 +153,10 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std } #ifndef WITHOUT_TEXT_LOG -void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_priority) +void OwnSplitChannel::addTextLog(std::shared_ptr> log_queue, int max_priority) { std::lock_guard lock(text_log_mutex); - text_log = log; + text_log = log_queue; text_log_max_priority.store(max_priority, std::memory_order_relaxed); } #endif diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 80305c1ccee..ab86c32f85a 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -10,7 +10,9 @@ #ifndef WITHOUT_TEXT_LOG namespace DB { - class TextLog; + template class SystemLogQueue; + struct TextLogElement; + using FooBar = SystemLogQueue; } #endif @@ -31,7 +33,7 @@ public: void addChannel(Poco::AutoPtr channel, const std::string & name); #ifndef WITHOUT_TEXT_LOG - void addTextLog(std::shared_ptr log, int max_priority); + void addTextLog(std::shared_ptr log_queue, int max_priority); #endif void setLevel(const std::string & name, int level); @@ -48,7 +50,7 @@ private: std::mutex text_log_mutex; #ifndef WITHOUT_TEXT_LOG - std::weak_ptr text_log; + std::weak_ptr text_log; std::atomic text_log_max_priority = -1; #endif }; From 3649f104444076f7f5cc232fae8940cde8025d46 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 13 Jul 2023 05:19:18 +0000 Subject: [PATCH 171/473] Remove spaces --- src/Common/SystemLogBase.cpp | 4 +--- src/Common/SystemLogBase.h | 2 +- src/Interpreters/TextLog.h | 4 +--- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 28dc5ec2666..2d2c3e2053f 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -73,9 +73,7 @@ void SystemLogBase::startup() { std::lock_guard lock(queue->mutex); std::cout << "void ISystemLog::startup()" << std::endl; - saving_thread = std::make_unique([this] { - std::cout << "void ISystemLog::ThreadFromGlobalPool()" << std::endl; - savingThreadFunction(); }); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); } static thread_local bool recursive_add_call = false; diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index cb6003c94ef..221d9946b48 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -67,7 +67,7 @@ public: virtual void savingThreadFunction() = 0; protected: - std::unique_ptr saving_thread; + std::unique_ptr saving_thread; }; template diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 9c761f9d865..33c38da2f8f 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -46,7 +46,7 @@ public: const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_); - + static std::shared_ptr> getLogQueue() { static std::shared_ptr> queue = std::make_shared>(); @@ -54,6 +54,4 @@ public: } }; - - } From 220c0255abea1c4fbb5dd66a4fda350be9aa9023 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 13 Jul 2023 08:15:50 +0000 Subject: [PATCH 172/473] Remove mutex --- src/Common/SystemLogBase.cpp | 1 - src/Loggers/Loggers.cpp | 10 ++++++---- src/Loggers/OwnSplitChannel.cpp | 6 +----- src/Loggers/OwnSplitChannel.h | 2 -- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 2d2c3e2053f..09f4c050f15 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -72,7 +72,6 @@ template void SystemLogBase::startup() { std::lock_guard lock(queue->mutex); - std::cout << "void ISystemLog::startup()" << std::endl; saving_thread = std::make_unique([this] { savingThreadFunction(); }); } diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 96793ee5e0a..2fda836e7b7 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -37,7 +37,6 @@ static std::string createDirectory(const std::string & file) void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { - auto current_logger = config.getString("logger", ""); if (config_logger.has_value() && *config_logger == current_logger) return; @@ -51,9 +50,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split = new DB::OwnSplitChannel(); #ifndef WITHOUT_TEXT_LOG - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); - split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + if (config.has("text_log")) + { + String text_log_level_str = config.getString("text_log.level", ""); + int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + } #endif auto log_level_string = config.getString("logger.level", "trace"); diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 7661996ea33..17806153905 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -136,10 +136,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) elem.message_format_string = msg.getFormatString(); std::shared_ptr> text_log_locked{}; - { - std::lock_guard lock(text_log_mutex); - text_log_locked = text_log.lock(); - } + text_log_locked = text_log.lock(); if (text_log_locked) text_log_locked->add(elem); } @@ -155,7 +152,6 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std #ifndef WITHOUT_TEXT_LOG void OwnSplitChannel::addTextLog(std::shared_ptr> log_queue, int max_priority) { - std::lock_guard lock(text_log_mutex); text_log = log_queue; text_log_max_priority.store(max_priority, std::memory_order_relaxed); } diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index ab86c32f85a..155d0872465 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -47,8 +47,6 @@ private: using ExtendedChannelPtrPair = std::pair; std::map channels; - std::mutex text_log_mutex; - #ifndef WITHOUT_TEXT_LOG std::weak_ptr text_log; std::atomic text_log_max_priority = -1; From bc392a607321093f8fa4e12b99289ae4c113fd80 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 13 Jul 2023 09:55:35 +0000 Subject: [PATCH 173/473] enable flag in sqllogic --- tests/sqllogic/connection.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sqllogic/connection.py b/tests/sqllogic/connection.py index ca03839fc35..0033c29c41c 100644 --- a/tests/sqllogic/connection.py +++ b/tests/sqllogic/connection.py @@ -62,7 +62,7 @@ def default_clickhouse_odbc_conn_str(): return str( OdbcConnectingArgs.create_from_kw( dsn="ClickHouse DSN (ANSI)", - Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1", + Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1", ) ) From 3a54182f778ca96cb107f7d934719553447a2733 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 13:55:55 +0200 Subject: [PATCH 174/473] Fix test_extreme_deduplication flakiness This is part of **"Why my PR is not merged?"** patch set series. Since #50107 there one more tunable that can increase the cleanup period - max_cleanup_delay_period. Plus, increase the delay for the cleanup thread, otherwise it is too fragile. Signed-off-by: Azat Khuzhin --- .../test_extreme_deduplication/configs/conf.d/merge_tree.xml | 1 + tests/integration/test_extreme_deduplication/test.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml b/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml index d9062e8da20..6f1d05d125e 100644 --- a/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml +++ b/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml @@ -4,6 +4,7 @@ 1 1 0 + 1 1 diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 71f783d37c9..9604809f0b5 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -50,7 +50,7 @@ def test_deduplication_window_in_seconds(started_cluster): assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n") # wait clean thread - time.sleep(2) + time.sleep(5) assert ( TSV.toMat( From ba57c84db3c778eb4d0fe4e5a19b7615ab0772be Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 3 Jul 2023 11:40:11 +0800 Subject: [PATCH 175/473] bug fix csv input field type mismatch --- docs/en/interfaces/formats.md | 2 + .../operations/settings/settings-formats.md | 29 ++++++++++++ src/Core/Settings.h | 2 + src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 2 + .../Formats/Impl/CSVRowInputFormat.cpp | 47 ++++++++++++++++--- ...11_csv_input_field_type_mismatch.reference | 3 ++ .../02811_csv_input_field_type_mismatch.sh | 13 +++++ .../data_csv/csv_with_diff_field_types.csv | 3 ++ 9 files changed, 97 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed2f010a632..ab6e514b4b2 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,6 +472,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. +- [input_format_csv_allow_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_set_default_if_deserialize_failed) - Allow to set default value to column if the csv input field deserialize failed. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 3eea5ef4ad9..bb2159210d4 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,6 +969,35 @@ Result a b ``` +### input_format_csv_allow_check_deserialize_result {#input_format_csv_allow_check_deserialize_result} + +Allow to use whitespace or tab as field delimiter in CSV strings. + +Default value: `false`. + +### input_format_csv_allow_set_column_default_value_if_deserialize_failed {#input_format_csv_allow_set_column_default_value_if_deserialize_failed} + +Allow to set default value to column if the csv input field deserialize failed + +Default value: `false`. + +**Examples** + +Query + +```bash +echo 'a,b,c' > 1.txt +./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +./clickhouse local -q "select * from test_tbl" +``` + +Result + +```text +a 0 1971-01-01 +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..b67745d1f59 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,6 +872,8 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ + M(Bool, input_format_csv_allow_check_deserialize_result, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ + M(Bool, input_format_csv_allow_set_column_default_value_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 182abc84ffe..8a2a7e97941 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,6 +73,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; + format_settings.csv.allow_check_deserialize = settings.input_format_csv_allow_check_deserialize; + format_settings.csv.allow_set_default_if_deserialize_failed = settings.input_format_csv_allow_set_default_if_deserialize_failed; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index dd4608227d0..50ebdc1c67f 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,6 +141,8 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; + bool allow_check_deserialize = false; + bool allow_set_default_if_deserialize_failed=false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 79ce2549b4d..8c4660fe4f9 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -315,14 +316,48 @@ bool CSVFormatReader::readField( return false; } - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + BufferBase::Position pos_start = buf->position(); + size_t col_size = column.size(); + try { - /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + if (format_settings.csv.allow_check_deserialize_result) + { + std::string field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromMemory tmp(field); + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); + else + serialization->deserializeTextCSV(column, tmp, format_settings); + if (column.size() == col_size + 1 && field.size() > 0 && tmp.position() != tmp.buffer().end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); + } + else + { + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + { + /// If value is null but type is not nullable then use default value instead. + return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + } + /// Read the column normally. + serialization->deserializeTextCSV(column, *buf, format_settings); + } + } + catch (Exception & e) + { + LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); + if (format_settings.csv.allow_set_column_default_value_if_deserialize_failed) + { + // Reset the column and buffer position, then skip the field and set column default value. + if (column.size() == col_size + 1) + column.popBack(1); + buf->position() = pos_start; + skipField(); + column.insertDefault(); + } + else + throw; } - - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); return true; } diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference new file mode 100644 index 00000000000..c5ee611a230 --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -0,0 +1,3 @@ +a 1 2023-03-14 +a 0 1970-01-01 +c 1 1970-01-01 diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh new file mode 100644 index 00000000000..246905a5b18 --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" +$CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "select * from test_tbl" +$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv new file mode 100644 index 00000000000..464172c515c --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv @@ -0,0 +1,3 @@ +a,1,2023-03-14 +a,b,c +c,1,a \ No newline at end of file From 2720e9177d9be212e469ee5339decd8dda05a58d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 13:29:45 +0200 Subject: [PATCH 176/473] Fix flakiness of test_keeper_s3_snapshot This is part of "Why my PR is not merged?" patch set series. Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/Dockerfile | 1 + .../test_keeper_s3_snapshot/test.py | 29 ++++++++++++------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d6c127c8421..0d1fa00b214 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -98,6 +98,7 @@ RUN python3 -m pip install --no-cache-dir \ redis \ requests-kerberos \ tzlocal==2.1 \ + retry \ urllib3 # Hudi supports only spark 3.3.*, not 3.4 diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 3e19bc4822c..84ffc964621 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -1,6 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster from time import sleep +from retry import retry from kazoo.client import KazooClient @@ -88,15 +89,19 @@ def test_s3_upload(started_cluster): for obj in list(cluster.minio_client.list_objects("snapshots")) ] - saved_snapshots = get_saved_snapshots() - assert set(saved_snapshots) == set( - [ - "snapshot_50.bin.zstd", - "snapshot_100.bin.zstd", - "snapshot_150.bin.zstd", - "snapshot_200.bin.zstd", - ] - ) + # Keeper sends snapshots asynchornously, hence we need to retry. + @retry(AssertionError, tries=10, delay=2) + def _check_snapshots(): + assert set(get_saved_snapshots()) == set( + [ + "snapshot_50.bin.zstd", + "snapshot_100.bin.zstd", + "snapshot_150.bin.zstd", + "snapshot_200.bin.zstd", + ] + ) + + _check_snapshots() destroy_zk_client(node1_zk) node1.stop_clickhouse(kill=True) @@ -108,9 +113,11 @@ def test_s3_upload(started_cluster): for _ in range(200): node2_zk.create("/test", sequence=True) - saved_snapshots = get_saved_snapshots() + @retry(AssertionError, tries=10, delay=2) + def _check_snapshots_without_quorum(): + assert len(get_saved_snapshots()) > 4 - assert len(saved_snapshots) > 4 + _check_snapshots_without_quorum() success_upload_message = "Successfully uploaded" assert node2.contains_in_log(success_upload_message) or node3.contains_in_log( From b2665031dcbaabb2a16b36459eee2b961d8f81f7 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 5 Jul 2023 11:55:34 +0800 Subject: [PATCH 177/473] review fix --- docs/en/interfaces/formats.md | 4 ++-- docs/en/operations/settings/settings-formats.md | 6 +++--- src/Core/Settings.h | 4 ++-- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 4 ++-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 4 ++-- .../0_stateless/02811_csv_input_field_type_mismatch.sh | 2 +- 7 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ab6e514b4b2..2207a6f0442 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. -- [input_format_csv_allow_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_set_default_if_deserialize_failed) - Allow to set default value to column if the csv input field deserialize failed. Default value - `false`. +- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. +- [input_format_csv_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialize_failed) - Set default value to column if the csv input field deserialize failed. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index bb2159210d4..b74504ac5fd 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,13 +969,13 @@ Result a b ``` -### input_format_csv_allow_check_deserialize_result {#input_format_csv_allow_check_deserialize_result} +### input_format_csv_allow_check_deserialize {#input_format_csv_allow_check_deserialize} Allow to use whitespace or tab as field delimiter in CSV strings. Default value: `false`. -### input_format_csv_allow_set_column_default_value_if_deserialize_failed {#input_format_csv_allow_set_column_default_value_if_deserialize_failed} +### input_format_csv_set_default_if_deserialize_failed {#input_format_csv_set_default_if_deserialize_failed} Allow to set default value to column if the csv input field deserialize failed @@ -988,7 +988,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b67745d1f59..5ea355c61c2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,8 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_deserialize_result, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ - M(Bool, input_format_csv_allow_set_column_default_value_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ + M(Bool, input_format_csv_allow_check_deserialize, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ + M(Bool, input_format_csv_set_default_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8a2a7e97941..ac7ceb7e791 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -74,7 +74,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; format_settings.csv.allow_check_deserialize = settings.input_format_csv_allow_check_deserialize; - format_settings.csv.allow_set_default_if_deserialize_failed = settings.input_format_csv_allow_set_default_if_deserialize_failed; + format_settings.csv.set_default_if_deserialize_failed = settings.input_format_csv_set_default_if_deserialize_failed; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 50ebdc1c67f..e9de7a51108 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,8 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_deserialize = false; - bool allow_set_default_if_deserialize_failed=false; + bool allow_check_deserialize=false; + bool set_default_if_deserialize_failed=false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8c4660fe4f9..a0709121d13 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -320,7 +320,7 @@ bool CSVFormatReader::readField( size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_deserialize_result) + if (format_settings.csv.allow_check_deserialize) { std::string field; readCSVField(field, *buf, format_settings.csv); @@ -329,7 +329,7 @@ bool CSVFormatReader::readField( SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); else serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && tmp.position() != tmp.buffer().end()) + if (column.size() == col_size + 1 && field.size() > 0 && !tm.eof()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); } else diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index 246905a5b18..df736ea6792 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -8,6 +8,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" $CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file From 4a10c7286e9fa95694284968be1f39fffcb29b6c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 13 Jul 2023 12:41:17 +0000 Subject: [PATCH 178/473] Add name to logger in queue --- src/Common/SystemLogBase.cpp | 19 ++++++++++++++----- src/Common/SystemLogBase.h | 5 ++++- src/Interpreters/SystemLog.cpp | 4 ++-- src/Interpreters/TextLog.h | 2 +- src/Loggers/Loggers.cpp | 17 ++++++++--------- 5 files changed, 29 insertions(+), 18 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 09f4c050f15..8285f133d73 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -39,12 +39,14 @@ namespace ISystemLog::~ISystemLog() = default; template -SystemLogBase::SystemLogBase(std::shared_ptr> ex_queue) +SystemLogBase::SystemLogBase( + const String & name_, + std::shared_ptr> queue_) { - if (ex_queue) - queue = ex_queue; + if (queue_) + queue = queue_; else - queue = std::make_shared>(); + queue = std::make_shared>(name_); } template @@ -75,6 +77,11 @@ void SystemLogBase::startup() saving_thread = std::make_unique([this] { savingThreadFunction(); }); } +template +SystemLogQueue::SystemLogQueue(const String & name_) + : log(&Poco::Logger::get(name_)) +{} + static thread_local bool recursive_add_call = false; template @@ -92,7 +99,6 @@ void SystemLogQueue::add(const LogElement & element) /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; - /// Should not log messages under mutex. bool queue_is_half_full = false; @@ -194,4 +200,7 @@ void SystemLogBase::flush(bool force) #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) +#define INSTANTIATE_SYSTEM_LOG_BASE2(ELEMENT) template class SystemLogQueue; +SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE2) + } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 221d9946b48..3b1b848369b 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -74,6 +74,7 @@ template class SystemLogQueue { public: + SystemLogQueue(const String & name_); void add(const LogElement & element); size_t size() const { return queue.size(); } @@ -109,7 +110,9 @@ class SystemLogBase : public ISystemLog public: using Self = SystemLogBase; - SystemLogBase(std::shared_ptr> ex_queue = nullptr); + SystemLogBase( + const String & name_, + std::shared_ptr> queue_ = nullptr); /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index cbf355d020a..b77cb2311d5 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -333,8 +333,8 @@ SystemLog::SystemLog( const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_, - std::shared_ptr> ex_queue) - : Base(ex_queue) + std::shared_ptr> queue_) + : Base(database_name_ + "." + table_name_, queue_) , WithContext(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_) diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 33c38da2f8f..8390259e147 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -49,7 +49,7 @@ public: static std::shared_ptr> getLogQueue() { - static std::shared_ptr> queue = std::make_shared>(); + static std::shared_ptr> queue = std::make_shared>("text_log"); return queue; } }; diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 2fda836e7b7..fa143440cc2 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -49,15 +49,6 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// Use extended interface of Channel for more comprehensive logging. split = new DB::OwnSplitChannel(); -#ifndef WITHOUT_TEXT_LOG - if (config.has("text_log")) - { - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); - split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); - } -#endif - auto log_level_string = config.getString("logger.level", "trace"); /// different channels (log, console, syslog) may have different loglevels configured @@ -258,6 +249,14 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log } } } +#ifndef WITHOUT_TEXT_LOG + if (config.has("text_log")) + { + String text_log_level_str = config.getString("text_log.level", ""); + int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + } +#endif } void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger) From 96912b12bcd4e6ae453ebc7ad950db6f5e83a64b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 13 Jul 2023 13:08:36 +0000 Subject: [PATCH 179/473] forgot to commit --- .../02813_create_index_noop.reference | 0 .../0_stateless/02813_create_index_noop.sql | 1000 +++++++++++++++++ 2 files changed, 1000 insertions(+) create mode 100644 tests/queries/0_stateless/02813_create_index_noop.reference create mode 100644 tests/queries/0_stateless/02813_create_index_noop.sql diff --git a/tests/queries/0_stateless/02813_create_index_noop.reference b/tests/queries/0_stateless/02813_create_index_noop.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02813_create_index_noop.sql b/tests/queries/0_stateless/02813_create_index_noop.sql new file mode 100644 index 00000000000..3d65f81af9d --- /dev/null +++ b/tests/queries/0_stateless/02813_create_index_noop.sql @@ -0,0 +1,1000 @@ +SET allow_create_index_without_type=0; +CREATE INDEX idx_tab1_0 on tab1 (col0); -- { serverError INCORRECT_QUERY } +SET allow_create_index_without_type=1; +CREATE INDEX idx_tab1_0 on tab1 (col0); +CREATE INDEX idx_tab1_1 on tab1 (col1); +CREATE INDEX idx_tab1_3 on tab1 (col3); +CREATE INDEX idx_tab1_4 on tab1 (col4); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col0); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col1 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col4,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col3 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col1); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col0,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col1,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col3 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col4); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col0 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col3 DESC,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col3,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col0); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col1,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3 DESC,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col0 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col4 DESC,col3,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col4,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0 DESC,col4 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col4); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col3 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col3,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col0); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col1 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col3 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col1); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col3,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col3); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col1 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col4); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col0,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col1 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4 DESC,col1 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col0,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col4 DESC,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col4 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col0 DESC,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col1,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col4,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col3); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col0,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col3,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col4); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col0); +CREATE INDEX idx_tab2_4 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_4 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1); +CREATE INDEX idx_tab2_4 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col3); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col0); +CREATE INDEX idx_tab2_5 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_5 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col4); +CREATE INDEX idx_tab2_5 ON tab2 (col1); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col0); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col4,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3); +CREATE INDEX idx_tab2_5 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col4); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col0,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col3,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col0 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col3,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col0,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col1,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1,col0 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col0,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col1,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col3,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1 DESC,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col3,col1,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col3,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col0); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col1,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col4 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col0,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col3,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col1); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col3 DESC,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col3); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col0,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col1,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col0); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col1); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col3); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0 DESC,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col3,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col4); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col0 DESC,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col3,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col0); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col1); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col1 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col3); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col0,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col4,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col4); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_4 ON tab3 (col0); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4 DESC,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1); +CREATE INDEX idx_tab3_4 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC,col1,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC,col4,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3); +CREATE INDEX idx_tab3_4 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC,col0 DESC,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col4); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col0,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col1,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col1,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col1); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col1,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col3); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col0 DESC,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col3,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col4 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col1,col4 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col4,col1,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col3,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col1); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col3 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col4,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col1 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col4 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col3); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0 DESC,col1,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col1 DESC,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col4,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col0,col1 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col3 DESC,col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col4); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col0,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col0); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col1,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col3,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col1); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col3,col4 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col4,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col3); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col1 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col0 DESC,col3 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col0,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col3 DESC,col0,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col0); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col4,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col0,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col0,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col3); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col3,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col0,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col0,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_3 ON tab4 (col0); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col3,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col4); +CREATE INDEX idx_tab4_3 ON tab4 (col1); +CREATE INDEX idx_tab4_3 ON tab4 (col1,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC,col1,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col3,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col3,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col4); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col0); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col1,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col4); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1); +CREATE INDEX idx_tab4_4 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3); +CREATE INDEX idx_tab4_4 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col4); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col3 DESC,col0); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col0); +CREATE INDEX idx_tab4_5 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1); +CREATE INDEX idx_tab4_5 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1,col4); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col1); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4 DESC,col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col4); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col1 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col3); \ No newline at end of file From 7e1b5d899744bbd1162e0bb4ba22851c576068ff Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 17:21:23 +0200 Subject: [PATCH 180/473] updated docs for toYearWeek() --- .../functions/date-time-functions.md | 14 +++++++++----- .../functions/date-time-functions.md | 15 +++++++++------ 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3f61e7a214d..4717813a56d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -694,10 +694,14 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we Returns year and week for a date. The year in the result may be different from the year in the date argument for the first and the last week of the year. -The mode argument works exactly like the mode argument to `toWeek()`. For the single-argument syntax, a mode value of 0 is used. +The mode argument works like the mode argument to `toWeek()`. For the single-argument syntax, a mode value of 0 is used. `toISOYear()` is a compatibility function that is equivalent to `intDiv(toYearWeek(date,3),100)`. +:::warning +The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. + + **Syntax** ``` sql @@ -707,13 +711,13 @@ toYearWeek(t[, mode[, timezone]]) **Example** ``` sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; +SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9, toYearWeek(toDate('2022-01-01')) AS prev_yearWeek; ``` ``` text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ +┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┬─prev_yearWeek─┐ +│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ 202152 │ +└────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` ## age diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 17ab04b7799..d251163df8d 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -599,24 +599,27 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we ## toYearWeek(date[,mode]) {#toyearweek} Возвращает год и неделю для даты. Год в результате может отличаться от года в аргументе даты для первой и последней недели года. -Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если mode не задан, используется режим 0. +Аргумент mode работает так же, как аргумент mode [toWeek()](#toweek), значение mode по умолчанию -- `0`. -`toISOYear() ` эквивалентно `intDiv(toYearWeek(date,3),100)`. +`toISOYear() ` эквивалентно `intDiv(toYearWeek(date,3),100)` + +:::warning +Однако, есть отличие в работе функций `toWeek()` и `toYearWeek()`. `toWeek()` возвращает номер недели в контексте заданного года, и в случае, когда `toWeek()` вернёт `0`, `toYearWeek()` вернёт значение, соответствующее последней неделе предыдущего года (см. `prev_yearWeek` в примере). **Пример** Запрос: ```sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; +SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9, toYearWeek(toDate('2022-01-01')) AS prev_yearWeek; ``` Результат: ```text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ +┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┬─prev_yearWeek─┐ +│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ 202152 │ +└────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` ## age From a87474cee1aeb924cb11c043308c36888f6a9a26 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 17:25:49 +0200 Subject: [PATCH 181/473] rm \n --- docs/en/sql-reference/functions/date-time-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4717813a56d..ddd6a367e31 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -701,7 +701,6 @@ The mode argument works like the mode argument to `toWeek()`. For the single-arg :::warning The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. - **Syntax** ``` sql From 0de7fb5c620eb6d7ee010f2053737f95577b77f8 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 13 Jul 2023 20:12:59 +0300 Subject: [PATCH 182/473] typofix1 --- docs/en/sql-reference/functions/date-time-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index fa5826fbdda..ce1a4f4d283 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -700,6 +700,7 @@ The mode argument works like the mode argument to `toWeek()`. For the single-arg :::warning The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. +::: **Syntax** From 54723d4822438cae9dc009c5b28e4f5fbadd1741 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 13 Jul 2023 20:14:06 +0300 Subject: [PATCH 183/473] typofix2 --- docs/ru/sql-reference/functions/date-time-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 8f31fcebf8b..4db8a1ec6f8 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -605,6 +605,7 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we :::warning Однако, есть отличие в работе функций `toWeek()` и `toYearWeek()`. `toWeek()` возвращает номер недели в контексте заданного года, и в случае, когда `toWeek()` вернёт `0`, `toYearWeek()` вернёт значение, соответствующее последней неделе предыдущего года (см. `prev_yearWeek` в примере). +::: **Пример** From f7505f0c7749d06f53dbfaf345f00127ca5ad012 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 19:41:45 +0200 Subject: [PATCH 184/473] update docs --- docs/en/sql-reference/data-types/datetime.md | 1 + docs/en/sql-reference/data-types/datetime64.md | 1 + .../sql-reference/functions/type-conversion-functions.md | 6 +++++- docs/ru/sql-reference/data-types/datetime.md | 1 + docs/ru/sql-reference/data-types/datetime64.md | 1 + .../sql-reference/functions/type-conversion-functions.md | 8 +++++++- 6 files changed, 16 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 0da273e01ad..fe279edb709 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -143,5 +143,6 @@ Time shifts for multiple days. Some pacific islands changed their timezone offse - [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) - [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) - [The `Date` data type](../../sql-reference/data-types/date.md) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 793691850b1..3b80e8b1a8b 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -119,6 +119,7 @@ FROM dt; - [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format) - [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) - [`DateTime` data type](../../sql-reference/data-types/datetime.md) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 214c885bc0e..36f40b37238 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -399,7 +399,11 @@ toDateTime(expr[, time_zone ]) - `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). - `time_zone` — Time zone. [String](/docs/en/sql-reference/data-types/string.md). -If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). +:::note +If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). +If `expr` is a [String](/docs/en/sql-reference/data-types/string.md), it may be interpreted as a Unix timestamp or as a string representation of date / date with time. +Thus, parsing of short numbers' string representations (up to 4 digits) is explicitly disabled due to ambiguity, e.g. a string `'1999'` may be both a year (an incomplete string representation of Date / DateTime) or a unix timestamp. Longer numeric strings are allowed. +::: **Returned value** diff --git a/docs/ru/sql-reference/data-types/datetime.md b/docs/ru/sql-reference/data-types/datetime.md index e8d4a3ee9fd..80d844a1713 100644 --- a/docs/ru/sql-reference/data-types/datetime.md +++ b/docs/ru/sql-reference/data-types/datetime.md @@ -122,6 +122,7 @@ FROM dt - [Настройка `date_time_input_format`](../../operations/settings/index.md#settings-date_time_input_format) - [Настройка `date_time_output_format`](../../operations/settings/index.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) - [Тип данных `DateTime64`](datetime64.md) diff --git a/docs/ru/sql-reference/data-types/datetime64.md b/docs/ru/sql-reference/data-types/datetime64.md index da2f81f4828..78ad43e4764 100644 --- a/docs/ru/sql-reference/data-types/datetime64.md +++ b/docs/ru/sql-reference/data-types/datetime64.md @@ -102,6 +102,7 @@ FROM dt; - [Настройка `date_time_input_format`](../../operations/settings/settings.md#settings-date_time_input_format) - [Настройка `date_time_output_format`](../../operations/settings/settings.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) - [Тип данных `DateTime`](datetime.md) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index d5e6246fe9e..e53104d8d71 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -284,7 +284,13 @@ toDateTime(expr[, time_zone ]) - `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). - `time_zone` — Часовой пояс. [String](/docs/ru/sql-reference/data-types/string.md). -Если `expr` является числом, оно интерпретируется как количество секунд от начала unix эпохи. +:::note +Если `expr` является числом, то оно интерпретируется как число секунд с начала Unix-эпохи (Unix Timestamp). + +Если же `expr` -- [строка (String)](/docs/ru/sql-reference/data-types/string.md), то оно может быть интерпретировано и как Unix Timestamp, и как строковое представление даты / даты со временем. +Ввиду неоднозначности запрещён парсинг строк длиной 4 и меньше. Так, строка `'1999'` могла бы представлять собой как год (неполное строковое представление даты или даты со временем), так и Unix Timestamp. +Строки длиной 5 символов и более не несут неоднозначности, а следовательно, их парсинг разрешён. +::: **Возвращаемое значение** From 2f17208792363f73db8fb750aa6e3e6f907bac37 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 20:38:17 +0200 Subject: [PATCH 185/473] Add missing --force for docker network prune (otherwise it is noop on CI) On CI there is no stdin attached, so this command simply do nothing, fix this. Fixes: https://github.com/ClickHouse/ClickHouse/pull/51969 Signed-off-by: Azat Khuzhin --- tests/integration/conftest.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 5933883f7b0..a4e25e653b3 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -45,7 +45,7 @@ def cleanup_environment(): logging.debug("Pruning Docker networks") run_and_check( - ["docker network prune"], + ["docker network prune --force"], shell=True, nothrow=True, ) From c6b8097090a8c418667c43997ba0be36b41acee4 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 14 Jul 2023 11:24:38 +0800 Subject: [PATCH 186/473] rebase main --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index a0709121d13..ada9d78711a 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -329,7 +329,7 @@ bool CSVFormatReader::readField( SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); else serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && !tm.eof()) + if (column.size() == col_size + 1 && field.size() > 0 && !tmp.eof()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); } else @@ -346,7 +346,7 @@ bool CSVFormatReader::readField( catch (Exception & e) { LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.allow_set_column_default_value_if_deserialize_failed) + if (format_settings.csv.set_default_if_deserialize_failed) { // Reset the column and buffer position, then skip the field and set column default value. if (column.size() == col_size + 1) From 575de8d497673f7db12ccaa893921957d6d6725e Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 14 Jul 2023 07:04:17 +0200 Subject: [PATCH 187/473] Fix typo last_removal_attemp_time --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/System/StorageSystemParts.cpp | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2c0cf37b3a5..52f4e385a0e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -500,7 +500,7 @@ public: mutable std::atomic removal_state = DataPartRemovalState::NOT_ATTEMPTED; - mutable std::atomic last_removal_attemp_time = 0; + mutable std::atomic last_removal_attempt_time = 0; protected: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ec8ce3f5e3d..e8dbf48c7a2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2187,7 +2187,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) const DataPartPtr & part = *it; - part->last_removal_attemp_time.store(time_now, std::memory_order_relaxed); + part->last_removal_attempt_time.store(time_now, std::memory_order_relaxed); /// Do not remove outdated part if it may be visible for some transaction if (!part->version.canBeRemoved()) diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 7399bd789a7..576b38c1584 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -119,7 +119,7 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"has_lightweight_delete", std::make_shared()}, - {"last_removal_attemp_time", std::make_shared()}, + {"last_removal_attempt_time", std::make_shared()}, {"removal_state", std::make_shared()}, } ) @@ -343,7 +343,7 @@ void StorageSystemParts::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(part->hasLightweightDelete()); if (columns_mask[src_index++]) - columns[res_index++]->insert(static_cast(part->last_removal_attemp_time.load(std::memory_order_relaxed))); + columns[res_index++]->insert(static_cast(part->last_removal_attempt_time.load(std::memory_order_relaxed))); if (columns_mask[src_index++]) columns[res_index++]->insert(getRemovalStateDescription(part->removal_state.load(std::memory_order_relaxed))); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 3c221d6a473..ed7fe3bac42 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -516,7 +516,7 @@ CREATE TABLE system.parts `creation_csn` UInt64, `removal_csn` UInt64, `has_lightweight_delete` UInt8, - `last_removal_attemp_time` DateTime, + `last_removal_attempt_time` DateTime, `removal_state` String, `bytes` UInt64, `marks_size` UInt64, From c4dabd1aae40acf7df982f49b887483ff1e1c3b8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 14 Jul 2023 13:53:29 +0800 Subject: [PATCH 188/473] alias Signed-off-by: Lloyd-Pottiger --- src/Functions/now.cpp | 1 + src/Functions/today.cpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index d3a94379a61..827b800a243 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -138,6 +138,7 @@ private: REGISTER_FUNCTION(Now) { factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerAlias("current_timestamp", NowOverloadResolver::name, FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index f106e3992a8..16a5b98d7ec 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -86,6 +86,8 @@ public: REGISTER_FUNCTION(Today) { factory.registerFunction(); + factory.registerAlias("current_date", TodayOverloadResolver::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("curdate", TodayOverloadResolver::name, FunctionFactory::CaseInsensitive); } } From e4cbece6dd7ce2dfbb4f6a57345a6a57d3716aa8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 14 Jul 2023 14:03:19 +0800 Subject: [PATCH 189/473] update docs Signed-off-by: Lloyd-Pottiger --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 40ede6e0b62..5c199dd6a72 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1130,7 +1130,7 @@ Result: └──────────────────────────────────────────────────────────────┘ ``` -## now +## now/current_timestamp Returns the current date and time at the moment of query analysis. The function is a constant expression. @@ -1259,7 +1259,7 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today +## today/curdate/current_date Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. From 8ea3bf4ade73ef3188026d5ac64db48ab7fa09a5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 14 Jul 2023 16:09:22 +0800 Subject: [PATCH 190/473] improve ch to arrow --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index c3685e813d3..899b84cc132 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -208,7 +208,7 @@ namespace DB const String & column_name, ColumnPtr & column, const DataTypePtr & column_type, - const PaddedPODArray * null_bytemap, + const PaddedPODArray *, arrow::ArrayBuilder * array_builder, String format_name, size_t start, @@ -231,7 +231,9 @@ namespace DB /// Start new array. components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); + + /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column + fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From f153d6aa3c69b5d7a6e6df87ffdb84b1632e5abe Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 14 Jul 2023 10:02:18 +0000 Subject: [PATCH 191/473] Fix shutdown --- src/Common/SystemLogBase.cpp | 13 +++++++------ src/Common/SystemLogBase.h | 8 ++++---- src/Loggers/OwnSplitChannel.h | 6 +++--- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 211bd457714..7373786e514 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -62,6 +62,7 @@ void SystemLogBase::stopFlushThread() return; is_shutdown = true; + queue->shutdown(); /// Tell thread to shutdown. queue->flush_event.notify_all(); @@ -105,8 +106,8 @@ void SystemLogQueue::add(const LogElement & element) { std::unique_lock lock(mutex); - // if (queue.is_shutdown) - // return; // TODO + if (is_shutdown) + return; if (queue.size() == DBMS_SYSTEM_LOG_QUEUE_SIZE / 2) { @@ -191,18 +192,18 @@ uint64_t SystemLogBase::notifyFlushImpl(bool force) uint64_t this_thread_requested_offset; { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); if (is_shutdown) return uint64_t(-1); - this_thread_requested_offset = queue_front_index + queue.size(); + this_thread_requested_offset = queue->queue_front_index + queue->queue.size(); // Publish our flush request, taking care not to overwrite the requests // made by other threads. is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + queue->requested_flush_up_to = std::max(queue->requested_flush_up_to, this_thread_requested_offset); - flush_event.notify_all(); + queue->flush_event.notify_all(); } LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index cf6efc68119..109334964d2 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -78,10 +78,8 @@ public: void add(const LogElement & element); size_t size() const { return queue.size(); } - void push_back(const LogElement & element) - { - queue.push_back(element); - } + //void push_back(const LogElement & element) { queue.push_back(element); } + void shutdown() { is_shutdown = true; } // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; @@ -101,7 +99,9 @@ public: // Logged overflow message at this queue front index uint64_t logged_queue_full_at_index = -1; +private: Poco::Logger * log; + bool is_shutdown = false; }; template diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 155d0872465..a6ee8af5b14 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -12,7 +12,7 @@ namespace DB { template class SystemLogQueue; struct TextLogElement; - using FooBar = SystemLogQueue; + using TextLogQueue = SystemLogQueue; } #endif @@ -33,7 +33,7 @@ public: void addChannel(Poco::AutoPtr channel, const std::string & name); #ifndef WITHOUT_TEXT_LOG - void addTextLog(std::shared_ptr log_queue, int max_priority); + void addTextLog(std::shared_ptr log_queue, int max_priority); #endif void setLevel(const std::string & name, int level); @@ -48,7 +48,7 @@ private: std::map channels; #ifndef WITHOUT_TEXT_LOG - std::weak_ptr text_log; + std::weak_ptr text_log; std::atomic text_log_max_priority = -1; #endif }; From 103a689edc1291e6b243c8000c8100b043caf8c6 Mon Sep 17 00:00:00 2001 From: therealnick233 Date: Fri, 14 Jul 2023 18:36:42 +0800 Subject: [PATCH 192/473] Fixed typo in combinators.md line 303, 'Jonh' -> 'John' --- docs/en/sql-reference/aggregate-functions/combinators.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index a395b350a55..18ff5073e3f 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -300,7 +300,7 @@ SELECT groupArrayResample(30, 75, 30)(name, age) FROM people Consider the results. -`Jonh` is out of the sample because he’s too young. Other people are distributed according to the specified age intervals. +`John` is out of the sample because he’s too young. Other people are distributed according to the specified age intervals. Now let’s count the total number of people and their average wage in the specified age intervals. From f7705fff58e871913028a1683c6d7b21a82b9c16 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 14 Jul 2023 12:45:05 +0200 Subject: [PATCH 193/473] Update README.md --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 1036e1a97e1..932a10be0ab 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -* [**v23.6 Release Webinar**](https://clickhouse.com/company/events/v23-6-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-06) - Jun 29 - 23.6 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**v23.7 Release Webinar**](https://clickhouse.com/company/events/v23-7-community-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-07) - Jul 27 - 23.7 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18 * [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19 * [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20 @@ -34,13 +34,13 @@ Also, keep an eye out for upcoming meetups around the world. Somewhere else you ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v23.4 Release Webinar**](https://www.youtube.com/watch?v=4rrf6bk_mOg) Faster Parquet Reading, Asynchonous Connections to Reoplicas, Trailing Comma before FROM, extractKeyValuePairs, integrations updates, and so much more! Watch it now! +* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.7, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) - ## Interested in joining ClickHouse and making it your full time job? + ## Interested in joining ClickHouse and making it your full-time job? -We are a globally diverse and distributed team, united behind a common goal of creating industry-leading, real-time analytics. Here, you will have an opportunity to solve some of the most cutting edge technical challenges and have direct ownership of your work and vision. If you are a contributor by nature, a thinker as well as a doer - we’ll definitely click! +We are a globally diverse and distributed team, united behind a common goal of creating industry-leading, real-time analytics. Here, you will have an opportunity to solve some of the most cutting-edge technical challenges and have direct ownership of your work and vision. If you are a contributor by nature, a thinker and a doer - we’ll definitely click! Check out our **current openings** here: https://clickhouse.com/company/careers From 132052fbf2404d8faa5e14fbc2cbd5f8e16bdc1f Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 29 Jun 2023 12:16:40 -0700 Subject: [PATCH 194/473] Implement partial support for endianness-independent serialization --- .../AggregateFunctionGroupArray.h | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 7a5e6a8cb2d..b5905105457 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -266,19 +266,20 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const auto & value = this->data(place).value; - size_t size = value.size(); + const size_t size = value.size(); writeVarUInt(size, buf); - buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); + for (const auto & element : value) + writeBinaryLittleEndian(element, buf); if constexpr (Trait::last) - DB::writeIntBinary(this->data(place).total_values, buf); + writeBinaryLittleEndian(this->data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::writeIntBinary(this->data(place).total_values, buf); + writeBinaryLittleEndian(this->data(place).total_values, buf); WriteBufferFromOwnString rng_buf; rng_buf << this->data(place).rng; - DB::writeStringBinary(rng_buf.str(), buf); + writeStringBinary(rng_buf.str(), buf); } } @@ -297,16 +298,17 @@ public: auto & value = this->data(place).value; value.resize_exact(size, arena); - buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); + for (auto & element : value) + readBinaryLittleEndian(element, buf); if constexpr (Trait::last) - DB::readIntBinary(this->data(place).total_values, buf); + readBinaryLittleEndian(this->data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::readIntBinary(this->data(place).total_values, buf); + readBinaryLittleEndian(this->data(place).total_values, buf); std::string rng_string; - DB::readStringBinary(rng_string, buf); + readStringBinary(rng_string, buf); ReadBufferFromString rng_buf(rng_string); rng_buf >> this->data(place).rng; } @@ -603,14 +605,14 @@ public: node->write(buf); if constexpr (Trait::last) - DB::writeIntBinary(data(place).total_values, buf); + writeBinaryLittleEndian(data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::writeIntBinary(data(place).total_values, buf); + writeBinaryLittleEndian(data(place).total_values, buf); WriteBufferFromOwnString rng_buf; rng_buf << data(place).rng; - DB::writeStringBinary(rng_buf.str(), buf); + writeStringBinary(rng_buf.str(), buf); } } @@ -636,13 +638,13 @@ public: value[i] = Node::read(buf, arena); if constexpr (Trait::last) - DB::readIntBinary(data(place).total_values, buf); + readBinaryLittleEndian(data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::readIntBinary(data(place).total_values, buf); + readBinaryLittleEndian(data(place).total_values, buf); std::string rng_string; - DB::readStringBinary(rng_string, buf); + readStringBinary(rng_string, buf); ReadBufferFromString rng_buf(rng_string); rng_buf >> data(place).rng; } From 0527a32282f7d9153eecbb9c740be18140f741c4 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 11:27:38 -0700 Subject: [PATCH 195/473] Implement platform-independent serialization for further aggregate functions --- .../AggregateFunctionBoundingRatio.h | 51 +++++++++++-------- .../AggregateFunctionDeltaSum.h | 16 +++--- .../AggregateFunctionDeltaSumTimestamp.h | 24 ++++----- src/AggregateFunctions/QuantileApprox.h | 28 +++++----- src/AggregateFunctions/ReservoirSampler.h | 12 ++--- src/Common/TransformEndianness.hpp | 6 +++ 6 files changed, 76 insertions(+), 61 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 935adbf2b7d..82e4f1122a8 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -67,29 +67,38 @@ struct AggregateFunctionBoundingRatioData } } - void serialize(WriteBuffer & buf) const - { - writeBinary(empty, buf); - - if (!empty) - { - writePODBinary(left, buf); - writePODBinary(right, buf); - } - } - - void deserialize(ReadBuffer & buf) - { - readBinary(empty, buf); - - if (!empty) - { - readPODBinary(left, buf); - readPODBinary(right, buf); - } - } + void serialize(WriteBuffer & buf) const; + void deserialize(ReadBuffer & buf); }; +template +inline void transformEndianness(AggregateFunctionBoundingRatioData::Point & p) +{ + transformEndianness(p.x); + transformEndianness(p.y); +} + +void AggregateFunctionBoundingRatioData::serialize(WriteBuffer & buf) const +{ + writeBinaryLittleEndian(empty, buf); + + if (!empty) + { + writeBinaryLittleEndian(left, buf); + writeBinaryLittleEndian(right, buf); + } +} + +void AggregateFunctionBoundingRatioData::deserialize(ReadBuffer & buf) +{ + readBinaryLittleEndian(empty, buf); + + if (!empty) + { + readBinaryLittleEndian(left, buf); + readBinaryLittleEndian(right, buf); + } +} class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper { diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.h b/src/AggregateFunctions/AggregateFunctionDeltaSum.h index 199d2706d3a..d64f949825a 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSum.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.h @@ -103,18 +103,18 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - writeIntBinary(this->data(place).sum, buf); - writeIntBinary(this->data(place).first, buf); - writeIntBinary(this->data(place).last, buf); - writePODBinary(this->data(place).seen, buf); + writeBinaryLittleEndian(this->data(place).sum, buf); + writeBinaryLittleEndian(this->data(place).first, buf); + writeBinaryLittleEndian(this->data(place).last, buf); + writeBinaryLittleEndian(this->data(place).seen, buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - readIntBinary(this->data(place).sum, buf); - readIntBinary(this->data(place).first, buf); - readIntBinary(this->data(place).last, buf); - readPODBinary(this->data(place).seen, buf); + readBinaryLittleEndian(this->data(place).sum, buf); + readBinaryLittleEndian(this->data(place).first, buf); + readBinaryLittleEndian(this->data(place).last, buf); + readBinaryLittleEndian(this->data(place).seen, buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h index 5ca07bb0bdf..5eeb1425afb 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h @@ -144,22 +144,22 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - writeIntBinary(this->data(place).sum, buf); - writeIntBinary(this->data(place).first, buf); - writeIntBinary(this->data(place).first_ts, buf); - writeIntBinary(this->data(place).last, buf); - writeIntBinary(this->data(place).last_ts, buf); - writePODBinary(this->data(place).seen, buf); + writeBinaryLittleEndian(this->data(place).sum, buf); + writeBinaryLittleEndian(this->data(place).first, buf); + writeBinaryLittleEndian(this->data(place).first_ts, buf); + writeBinaryLittleEndian(this->data(place).last, buf); + writeBinaryLittleEndian(this->data(place).last_ts, buf); + writeBinaryLittleEndian(this->data(place).seen, buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - readIntBinary(this->data(place).sum, buf); - readIntBinary(this->data(place).first, buf); - readIntBinary(this->data(place).first_ts, buf); - readIntBinary(this->data(place).last, buf); - readIntBinary(this->data(place).last_ts, buf); - readPODBinary(this->data(place).seen, buf); + readBinaryLittleEndian(this->data(place).sum, buf); + readBinaryLittleEndian(this->data(place).first, buf); + readBinaryLittleEndian(this->data(place).first_ts, buf); + readBinaryLittleEndian(this->data(place).last, buf); + readBinaryLittleEndian(this->data(place).last_ts, buf); + readBinaryLittleEndian(this->data(place).seen, buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/QuantileApprox.h b/src/AggregateFunctions/QuantileApprox.h index f58f1396fb4..6b2a6cf4398 100644 --- a/src/AggregateFunctions/QuantileApprox.h +++ b/src/AggregateFunctions/QuantileApprox.h @@ -233,35 +233,35 @@ public: void write(WriteBuffer & buf) const { - writeIntBinary(compress_threshold, buf); - writeFloatBinary(relative_error, buf); - writeIntBinary(count, buf); - writeIntBinary(sampled.size(), buf); + writeBinaryLittleEndian(compress_threshold, buf); + writeBinaryLittleEndian(relative_error, buf); + writeBinaryLittleEndian(count, buf); + writeBinaryLittleEndian(sampled.size(), buf); for (const auto & stats : sampled) { - writeFloatBinary(stats.value, buf); - writeIntBinary(stats.g, buf); - writeIntBinary(stats.delta, buf); + writeBinaryLittleEndian(stats.value, buf); + writeBinaryLittleEndian(stats.g, buf); + writeBinaryLittleEndian(stats.delta, buf); } } void read(ReadBuffer & buf) { - readIntBinary(compress_threshold, buf); - readFloatBinary(relative_error, buf); - readIntBinary(count, buf); + readBinaryLittleEndian(compress_threshold, buf); + readBinaryLittleEndian(relative_error, buf); + readBinaryLittleEndian(count, buf); size_t sampled_len = 0; - readIntBinary(sampled_len, buf); + readBinaryLittleEndian(sampled_len, buf); sampled.resize(sampled_len); for (size_t i = 0; i < sampled_len; ++i) { auto stats = sampled[i]; - readFloatBinary(stats.value, buf); - readIntBinary(stats.g, buf); - readIntBinary(stats.delta, buf); + readBinaryLittleEndian(stats.value, buf); + readBinaryLittleEndian(stats.g, buf); + readBinaryLittleEndian(stats.delta, buf); } } diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index 3d723d5aace..7409a3fa0dd 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -207,8 +207,8 @@ public: void read(DB::ReadBuffer & buf) { - DB::readIntBinary(sample_count, buf); - DB::readIntBinary(total_values, buf); + DB::readBinaryLittleEndian(sample_count, buf); + DB::readBinaryLittleEndian(total_values, buf); size_t size = std::min(total_values, sample_count); static constexpr size_t MAX_RESERVOIR_SIZE = 1_GiB; @@ -224,22 +224,22 @@ public: rng_buf >> rng; for (size_t i = 0; i < samples.size(); ++i) - DB::readBinary(samples[i], buf); + DB::readBinaryLittleEndian(samples[i], buf); sorted = false; } void write(DB::WriteBuffer & buf) const { - DB::writeIntBinary(sample_count, buf); - DB::writeIntBinary(total_values, buf); + DB::writeBinaryLittleEndian(sample_count, buf); + DB::writeBinaryLittleEndian(total_values, buf); DB::WriteBufferFromOwnString rng_buf; rng_buf << rng; DB::writeStringBinary(rng_buf.str(), buf); for (size_t i = 0; i < std::min(sample_count, total_values); ++i) - DB::writeBinary(samples[i], buf); + DB::writeBinaryLittleEndian(samples[i], buf); } private: diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 4d690d75d9e..0a9055dde15 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -59,4 +59,10 @@ inline void transformEndianness(std::pair & pair) transformEndianness(pair.first); transformEndianness(pair.second); } + +template +inline void transformEndianness(StrongTypedef & x) +{ + transformEndianness(x.toUnderType()); +} } From dff5d78126318782cc7f3935c38013b010695c07 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Sun, 18 Jun 2023 02:03:14 +0000 Subject: [PATCH 196/473] Add jemalloc support --- contrib/jemalloc-cmake/CMakeLists.txt | 4 +- .../internal/jemalloc_internal_defs.h.in | 435 ++++++++++++++++++ 2 files changed, 438 insertions(+), 1 deletion(-) create mode 100644 contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 20025dfc63e..9a4ad372be3 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64)) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X )) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) @@ -148,6 +148,8 @@ elseif (ARCH_PPC64LE) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_ppc64le") elseif (ARCH_RISCV64) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_riscv64") +elseif (ARCH_S390X) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_s390x") else () message (FATAL_ERROR "internal jemalloc: This arch is not supported") endif () diff --git a/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..531f2bca0c2 --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,435 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#define JEMALLOC_OVERRIDE___LIBC_CALLOC +#define JEMALLOC_OVERRIDE___LIBC_FREE +#define JEMALLOC_OVERRIDE___LIBC_MALLOC +#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN +#define JEMALLOC_OVERRIDE___LIBC_REALLOC +#define JEMALLOC_OVERRIDE___LIBC_VALLOC +#define JEMALLOC_OVERRIDE___LIBC_PVALLOC +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 64 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +#define JEMALLOC_HAVE_SECURE_GETENV + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 12 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 20 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +#define JEMALLOC_BIG_ENDIAN + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +#define JEMALLOC_ENABLE_CXX + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From efa78a377eaebd5a8c369f56732e63a51eb6fd07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:54:16 +0300 Subject: [PATCH 197/473] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 9a4ad372be3..ab4d9d46c63 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X )) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) From b2546c8bb524df19a376f925ec14d1a730b72768 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:54:34 +0300 Subject: [PATCH 198/473] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index ab4d9d46c63..f2b37d7efae 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) From f0f69a2e3a7dabb0169c785e34f6c4a6639ccd83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:55:57 +0300 Subject: [PATCH 199/473] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index f2b37d7efae..15e965ed841 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -17,17 +17,17 @@ if (NOT ENABLE_JEMALLOC) endif () if (NOT OS_LINUX) - message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") + message (WARNING "jemalloc support on non-Linux is EXPERIMENTAL") endif() if (OS_LINUX) - # ThreadPool select job randomly, and there can be some threads that had been - # performed some memory heavy task before and will be inactive for some time, - # but until it will became active again, the memory will not be freed since by - # default each thread has it's own arena, but there should be not more then + # ThreadPool select job randomly, and there can be some threads that have been + # performed some memory-heavy tasks before and will be inactive for some time, + # but until it becomes active again, the memory will not be freed since, by + # default, each thread has its arena, but there should be no more than # 4*CPU arenas (see opt.nareans description). # - # By enabling percpu_arena number of arenas limited to number of CPUs and hence + # By enabling percpu_arena number of arenas is limited to the number of CPUs, and hence # this problem should go away. # # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to @@ -38,7 +38,7 @@ if (OS_LINUX) else() set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000") endif() -# CACHE variable is empty, to allow changing defaults without necessity +# CACHE variable is empty to allow changing defaults without the necessity # to purge cache set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) @@ -174,7 +174,7 @@ target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1) # jemalloc provides support for two different libunwind flavors: the original HP libunwind and the one coming with gcc / g++ / libstdc++. # The latter is identified by `JEMALLOC_PROF_LIBGCC` and uses `_Unwind_Backtrace` method instead of `unw_backtrace`. -# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracing. +# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracking. # # ClickHouse has to provide `unw_backtrace` method by the means of [commit 8e2b31e](https://github.com/ClickHouse/libunwind/commit/8e2b31e766dd502f6df74909e04a7dbdf5182eb1). target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1) From 4f6d59ce7f2eaf032a8e833153da909b155bf3ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 05:57:46 +0200 Subject: [PATCH 200/473] tests: drop existing view in test_materialized_mysql_database This should avoid failures of other tests in case of some previous test failed Signed-off-by: Azat Khuzhin --- .../test_materialized_mysql_database/materialized_with_ddl.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8cf9e67bf63..73f6e11d7f8 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -992,6 +992,7 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SHOW TABLES FROM db FORMAT TSV", "t\n") clickhouse_node.query("SYSTEM STOP MERGES db.t") + clickhouse_node.query("DROP VIEW v IF EXISTS") clickhouse_node.query("CREATE VIEW v AS SELECT * FROM db.t") mysql_node.query("INSERT INTO db.t VALUES (1, 1), (2, 2)") mysql_node.query("DELETE FROM db.t WHERE a = 2;") From f762a0e61523c1e46eb690a59a2452e4e8d0f3af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 18:56:53 +0300 Subject: [PATCH 201/473] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 932a10be0ab..f0a7dbe2408 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ Also, keep an eye out for upcoming meetups around the world. Somewhere else you ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.7, one convenient video! Watch it now! +* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.6, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) From 254b0a65b3c82124d4053e6d4d6099ec23c8980e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 20:03:48 +0200 Subject: [PATCH 202/473] Update CCTZ --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index 5e05432420f..8529bcef5cd 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 5e05432420f9692418e2e12aff09859e420b14a2 +Subproject commit 8529bcef5cd996b7c0f4d7475286b76b5d126c4c From 32bc5866ea1b2622bbd0b18d92d748b4d6a4f86e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 20:04:58 +0200 Subject: [PATCH 203/473] Update build.md for clang-16 --- docs/en/development/build.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..dfec20be58e 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -42,7 +42,7 @@ sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test For other Linux distribution - check the availability of LLVM's [prebuild packages](https://releases.llvm.org/download.html). -As of April 2023, any version of Clang >= 15 will work. +As of April 2023, clang-16 or higher will work. GCC as a compiler is not supported. To build with a specific Clang version: @@ -86,8 +86,8 @@ The build requires the following components: - Git (used to checkout the sources, not needed for the build) - CMake 3.20 or newer -- Compiler: Clang 15 or newer -- Linker: lld 15 or newer +- Compiler: clang-16 or newer +- Linker: lld-16 or newer - Ninja - Yasm - Gawk From 94112895a160bb9eddc226aff55f05f56b55b143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 21:09:58 +0300 Subject: [PATCH 204/473] Add RISC-V 64 to the docs --- docs/en/development/build.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..d55f7d8cced 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -11,7 +11,8 @@ Supported platforms: - x86_64 - AArch64 -- Power9 (experimental) +- PowerPC 64 LE (experimental) +- RISC-V 64 (experimental) ## Building on Ubuntu From 6abdde12cafb74b45cf5cb650caac133606dab45 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 18:31:45 +0200 Subject: [PATCH 205/473] Print Zxid in keeper stat command in hex (so as ZooKeeper) Signed-off-by: Azat Khuzhin --- src/Coordination/FourLetterCommand.cpp | 18 ++++++++++++++++-- .../test_keeper_four_word_command/test.py | 4 ++-- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 0490213d296..d1ff03482b6 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -18,6 +18,20 @@ #include #include +namespace +{ + +String formatZxid(int64_t zxid) +{ + /// ZooKeeper print zxid in hex and + String hex = getHexUIntLowercase(zxid); + /// without leading zeros + trimLeft(hex, '0'); + return "0x" + hex; +} + +} + namespace DB { @@ -348,7 +362,7 @@ String ServerStatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", toString(keeper_info.last_zxid)); + write("Zxid", formatZxid(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); write("Node count", toString(keeper_info.total_nodes_count)); @@ -381,7 +395,7 @@ String StatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", toString(keeper_info.last_zxid)); + write("Zxid", formatZxid(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); write("Node count", toString(keeper_info.total_nodes_count)); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index b546d879c75..1d5bc6a6541 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -329,7 +329,7 @@ def test_cmd_srvr(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) > 10 + assert int(result["Zxid"], 16) > 10 assert result["Mode"] == "leader" assert result["Node count"] == "14" @@ -369,7 +369,7 @@ def test_cmd_stat(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) >= 10 + assert int(result["Zxid"], 16) >= 10 assert result["Mode"] == "leader" assert result["Node count"] == "14" From 9307e60c20fd634da1e88f2e839a30c963e7f4c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 23:21:26 +0300 Subject: [PATCH 206/473] Revert "Millisecond and microsecond support in date_diff / age functions" --- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 2 - src/Core/DecimalFunctions.h | 6 +- src/Functions/DateTimeTransforms.h | 71 ++---------- src/Functions/TransformDateTime64.h | 7 +- src/Functions/dateDiff.cpp | 58 +++------- src/Functions/toStartOfInterval.cpp | 1 + ...0479_date_and_datetime_to_number.reference | 1 - .../00479_date_and_datetime_to_number.sql | 1 - .../02160_special_functions.reference | 10 -- .../0_stateless/02160_special_functions.sql | 12 -- .../02477_age_datetime64.reference | 106 ------------------ .../0_stateless/02477_age_datetime64.sql | 65 ----------- 14 files changed, 29 insertions(+), 323 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 40ede6e0b62..3f61e7a214d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -718,7 +718,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond. +Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second. E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit. For an alternative to `age`, see function `date\_diff`. @@ -734,8 +734,6 @@ age('unit', startdate, enddate, [timezone]) - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: - - `microsecond` (possible abbreviations: `us`, `u`) - - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) @@ -811,8 +809,6 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_ - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: - - `microsecond` (possible abbreviations: `us`, `u`) - - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 864ae6e905e..17ab04b7799 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -621,7 +621,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду. +Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду. Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`. **Синтаксис** @@ -635,8 +635,6 @@ age('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: - - `microsecond` (возможные сокращения: `us`, `u`) - - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) @@ -710,8 +708,6 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: - - `microsecond` (возможные сокращения: `us`, `u`) - - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index e4b70322477..53dadc23c6d 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -643,8 +643,6 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。 可能的值: - - `microsecond` - - `millisecond` - `second` - `minute` - `hour` diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 17d95650730..357cff2c541 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -48,11 +48,7 @@ inline auto scaleMultiplier(UInt32 scale) /** Components of DecimalX value: * whole - represents whole part of decimal, can be negative or positive. - * fractional - for fractional part of decimal. - * - * 0.123 represents 0 / 0.123 - * -0.123 represents 0 / -0.123 - * -1.123 represents -1 / 0.123 + * fractional - for fractional part of decimal, always positive. */ template struct DecimalComponents diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a1c880f6956..510a88db2b6 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -19,9 +19,6 @@ namespace DB { -static constexpr auto microsecond_multiplier = 1000000; -static constexpr auto millisecond_multiplier = 1000; - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -1380,36 +1377,6 @@ struct ToRelativeSecondNumImpl using FactorTransform = ZeroTransform; }; -template -struct ToRelativeSubsecondNumImpl -{ - static constexpr auto name = "toRelativeSubsecondNumImpl"; - - static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) - { - static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000); - if (scale == scale_multiplier) - return t.value; - if (scale > scale_multiplier) - return t.value / (scale / scale_multiplier); - return t.value * (scale_multiplier / scale); - } - static inline Int64 execute(UInt32 t, const DateLUTImpl &) - { - return t * scale_multiplier; - } - static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) - { - return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; - } - static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone) - { - return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); - } - - using FactorTransform = ZeroTransform; -}; - struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; @@ -1509,47 +1476,25 @@ struct ToYYYYMMDDhhmmssImpl using FactorTransform = ZeroTransform; }; -struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeComponents -{ - UInt16 millisecond; - UInt16 microsecond; -}; - struct ToDateTimeComponentsImpl { static constexpr auto name = "toDateTimeComponents"; - static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(Int64 t, const DateLUTImpl & time_zone) { - auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); - - if (t.value < 0 && components.fractional) - { - components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional; - --components.whole; - } - Int64 fractional = components.fractional; - if (scale_multiplier > microsecond_multiplier) - fractional = fractional / (scale_multiplier / microsecond_multiplier); - else if (scale_multiplier < microsecond_multiplier) - fractional = fractional * (microsecond_multiplier / scale_multiplier); - - constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier; - UInt16 millisecond = static_cast(fractional / divider); - UInt16 microsecond = static_cast(fractional % divider); - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond}; + return time_zone.toDateTimeComponents(t); } - static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(UInt32 t, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast(t)), 0, 0}; + return time_zone.toDateTimeComponents(static_cast(t)); } - static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(Int32 d, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0}; + return time_zone.toDateTimeComponents(ExtendedDayNum(d)); } - static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(UInt16 d, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0}; + return time_zone.toDateTimeComponents(DayNum(d)); } using FactorTransform = ZeroTransform; diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index fcee2753066..3dab9efeb6b 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -5,7 +5,7 @@ namespace DB { -/** Transform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. +/** Tansform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. * * Depending on what overloads of Transform::execute() are available, when called with DateTime64 value, * invokes Transform::execute() with either: @@ -80,10 +80,7 @@ public: } else { - auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); - if (t.value < 0 && components.fractional) - --components.whole; - + const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); return wrapped_transform.execute(static_cast(components.whole), std::forward(args)...); } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 6bfbbb7c735..8361e9db166 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -174,13 +174,12 @@ public: { auto res = static_cast(transform_y.execute(y, timezone_y)) - static_cast(transform_x.execute(x, timezone_x)); - DateTimeComponentsWithFractionalPart a_comp; - DateTimeComponentsWithFractionalPart b_comp; + DateLUTImpl::DateTimeComponents a_comp; + DateLUTImpl::DateTimeComponents b_comp; Int64 adjust_value; - auto x_microseconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); - auto y_microseconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); - - if (x_microseconds <= y_microseconds) + auto x_seconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); + auto y_seconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); + if (x_seconds <= y_seconds) { a_comp = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, timezone_x); b_comp = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, timezone_y); @@ -193,16 +192,14 @@ public: adjust_value = 1; } - if constexpr (std::is_same_v>>) { if ((a_comp.date.month > b_comp.date.month) || ((a_comp.date.month == b_comp.date.month) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -213,9 +210,8 @@ public: || ((x_month_in_quarter == y_month_in_quarter) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -223,9 +219,8 @@ public: if ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -235,44 +230,25 @@ public: if ((x_day_of_week > y_day_of_week) || ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour)) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))) res += adjust_value; } else if constexpr (std::is_same_v>>) { - if ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))) - res += adjust_value; - } - else if constexpr (std::is_same_v>>) - { - if ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))) - res += adjust_value; - } - else if constexpr (std::is_same_v>>) - { - if (a_comp.microsecond > b_comp.microsecond) + if (a_comp.time.second > b_comp.time.second) res += adjust_value; } return res; @@ -397,10 +373,6 @@ public: impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); - else if (unit == "millisecond" || unit == "ms") - impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); - else if (unit == "microsecond" || unit == "us" || unit == "u") - impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} does not support '{}' unit", getName(), unit); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 48bf88cb14c..649242d0d86 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 168b733d702..1375ccb1542 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,4 +4,3 @@ 201707 20170721 20170721112233 -19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 1e35e99a802..71151690028 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,4 +4,3 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 5e7e3383d8d..3a1dcd88902 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,14 +33,4 @@ Hello 2021-01-01 1 1 -86400000 -172800000 -86461000 -86401299 -701 -701 -800 -60200201 -60 -10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 64919536be3..6d18e7d0d25 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,16 +41,4 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); -SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); - -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); - SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index c8c716e1e9a..3b4459dd26d 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,109 +111,3 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -5100200 -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); -5100200 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -5100 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); -5099 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -4 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -5 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); -4 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -3 -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -3 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); -1 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -16 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); -15 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -5 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); -4 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -8 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); -7 --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -2349 -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -2 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -5 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 889137395a3..1bed93991ca 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,68 +75,3 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); - --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); - -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); - -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); - -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); - -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); - -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); - -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); - -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); - --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file From cbe0a0e748cd0301292bb8079bd093e998d3031c Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 15 Jul 2023 03:04:56 +0200 Subject: [PATCH 207/473] upd threshold due to dt64 range change --- src/IO/ReadHelpers.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index b8ce162ec91..2636898c1b3 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1005,8 +1005,8 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } } } - /// 9908870400 is time_t value for 2184-01-01 UTC (a bit over the last year supported by DateTime64) - else if (whole >= 9908870400LL) + /// 10413792000 is time_t value for 2300-01-01 UTC (a bit over the last year supported by DateTime64) + else if (whole >= 10413792000LL) { /// Unix timestamp with subsecond precision, already scaled to integer. /// For disambiguation we support only time since 2001-09-09 01:46:40 UTC and less than 30 000 years in future. From 761cc7afa584438824d76ed2ed9385a0b340fc04 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 15 Jul 2023 03:17:18 +0200 Subject: [PATCH 208/473] add tests --- .../0_stateless/01802_toDateTime64_large_values.reference | 4 ++++ .../queries/0_stateless/01802_toDateTime64_large_values.sql | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01802_toDateTime64_large_values.reference b/tests/queries/0_stateless/01802_toDateTime64_large_values.reference index e60b1c30314..f3810cc3d4b 100644 --- a/tests/queries/0_stateless/01802_toDateTime64_large_values.reference +++ b/tests/queries/0_stateless/01802_toDateTime64_large_values.reference @@ -8,3 +8,7 @@ SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); 2205-12-12 12:12:12.000000 SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); 2205-12-12 12:12:12.000000 +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +2299-12-31 23:59:59.000 +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +2299-12-31 23:59:59.000 diff --git a/tests/queries/0_stateless/01802_toDateTime64_large_values.sql b/tests/queries/0_stateless/01802_toDateTime64_large_values.sql index d82d4433b2d..5c2e65188c3 100644 --- a/tests/queries/0_stateless/01802_toDateTime64_large_values.sql +++ b/tests/queries/0_stateless/01802_toDateTime64_large_values.sql @@ -4,4 +4,7 @@ SELECT toDateTime64('2205-12-12 12:12:12', 0, 'UTC'); SELECT toDateTime64('2205-12-12 12:12:12', 0, 'Asia/Istanbul'); SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); -SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); \ No newline at end of file +SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); + +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); \ No newline at end of file From 801d0955ececaa7141b3ed5a4210afab6eb19d1d Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jul 2023 12:08:40 +0300 Subject: [PATCH 209/473] Add tests --- tests/integration/helpers/cluster.py | 18 +++ tests/integration/parallel_skip.json | 6 +- tests/integration/test_dns_cache/test.py | 32 +++-- .../test_http_failover/__init__.py | 0 .../test_http_failover/configs/listen.xml | 1 + tests/integration/test_http_failover/test.py | 113 ++++++++++++++++++ 6 files changed, 152 insertions(+), 18 deletions(-) create mode 100644 tests/integration/test_http_failover/__init__.py create mode 100644 tests/integration/test_http_failover/configs/listen.xml create mode 100644 tests/integration/test_http_failover/test.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0614cbf0e0d..8548bc2b1b2 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3522,6 +3522,24 @@ class ClickHouseInstance: return error + def append_hosts(self, name, ip): + self.exec_in_container( + (["bash", "-c", "echo '{}' {} >> /etc/hosts".format(ip, name)]), + privileged=True, + user="root", + ) + + def set_hosts(self, hosts): + entries = ["127.0.0.1 localhost", "::1 localhost"] + for host in hosts: + entries.append(f"{host[0]} {host[1]}") + + self.exec_in_container( + ["bash", "-c", 'echo -e "{}" > /etc/hosts'.format("\\n".join(entries))], + privileged=True, + user="root", + ) + # Connects to the instance via HTTP interface, sends a query and returns both the answer and the error message # as a tuple (output, error). def http_query_and_get_answer_with_error( diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index d060218456a..407fe7d1b01 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -68,5 +68,9 @@ "test_server_reload/test.py::test_remove_postgresql_port", "test_server_reload/test.py::test_remove_tcp_port", - "test_keeper_map/test.py::test_keeper_map_without_zk" + "test_keeper_map/test.py::test_keeper_map_without_zk", + + "test_http_failover/test.py::test_url_destination_host_with_multiple_addrs", + "test_http_failover/test.py::test_url_invalid_hostname", + "test_http_failover/test.py::test_url_ip_change" ] diff --git a/tests/integration/test_dns_cache/test.py b/tests/integration/test_dns_cache/test.py index 98417930713..9c1c9797383 100644 --- a/tests/integration/test_dns_cache/test.py +++ b/tests/integration/test_dns_cache/test.py @@ -55,6 +55,13 @@ def cluster_without_dns_cache_update(): # node1 is a source, node2 downloads data # node2 has long dns_cache_update_period, so dns cache update wouldn't work def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): + # In this case we should manually set up the static DNS entries on the source host + # to exclude resplving addresses automatically added by docker. + # We use ipv6 for hosts, but resolved DNS entries may contain an unexpected ipv4 address. + node2.set_hosts([("2001:3984:3989::1:1111", "node1")]) + # drop DNS cache + node2.query("SYSTEM DROP DNS CACHE") + # First we check, that normal replication works node1.query( "INSERT INTO test_table_drop VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)" @@ -64,6 +71,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): # We change source node ip cluster.restart_instance_with_ip_change(node1, "2001:3984:3989::1:7777") + node2.set_hosts([("2001:3984:3989::1:7777", "node1")]) # Put some data to source node1 node1.query( @@ -163,17 +171,8 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update): assert_eq_with_retry(node4, "SELECT count(*) from test_table_update", "7") -def set_hosts(node, hosts): - new_content = "\\n".join(["127.0.0.1 localhost", "::1 localhost"] + hosts) - node.exec_in_container( - ["bash", "-c", 'echo -e "{}" > /etc/hosts'.format(new_content)], - privileged=True, - user="root", - ) - - def test_dns_cache_update(cluster_with_dns_cache_update): - set_hosts(node4, ["127.255.255.255 lost_host"]) + node4.set_hosts([("127.255.255.255", "lost_host")]) with pytest.raises(QueryRuntimeException): node4.query("SELECT * FROM remote('lost_host', 'system', 'one')") @@ -184,7 +183,7 @@ def test_dns_cache_update(cluster_with_dns_cache_update): with pytest.raises(QueryRuntimeException): node4.query("SELECT * FROM distributed_lost_host") - set_hosts(node4, ["127.0.0.1 lost_host"]) + node4.set_hosts([("127.0.0.1", "lost_host")]) # Wait a bit until dns cache will be updated assert_eq_with_retry( @@ -239,11 +238,10 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node): == "0\n" ) - set_hosts( - node, + node.set_hosts( [ - "127.255.255.255 node3", - "2001:3984:3989::1:88{}4 unknown_host".format(node_num), + ("127.255.255.255", "node3"), + (f"2001:3984:3989::1:88{node_num}4", "unknown_host"), ], ) @@ -260,7 +258,7 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node): node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) # now wrong addresses are cached - set_hosts(node, []) + node.set_hosts([]) retry_count = 60 if node_name == "node5": # client is not allowed to connect, so execute it directly in container to send query from localhost @@ -298,7 +296,7 @@ def test_host_is_drop_from_cache_after_consecutive_failures( # Note that the list of hosts in variable since lost_host will be there too (and it's dropped and added back) # dns_update_short -> dns_max_consecutive_failures set to 6 assert node4.wait_for_log_line( - "Cannot resolve host \\(InvalidHostThatDoesNotExist\\), error 0: Host not found." + "Code: 198. DB::Exception: Not found address of host: InvalidHostThatDoesNotExist." ) assert node4.wait_for_log_line( "Cached hosts not found:.*InvalidHostThatDoesNotExist**", diff --git a/tests/integration/test_http_failover/__init__.py b/tests/integration/test_http_failover/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_http_failover/configs/listen.xml b/tests/integration/test_http_failover/configs/listen.xml new file mode 100644 index 00000000000..3abb37d5da2 --- /dev/null +++ b/tests/integration/test_http_failover/configs/listen.xml @@ -0,0 +1 @@ +:: diff --git a/tests/integration/test_http_failover/test.py b/tests/integration/test_http_failover/test.py new file mode 100644 index 00000000000..41b55ef635c --- /dev/null +++ b/tests/integration/test_http_failover/test.py @@ -0,0 +1,113 @@ +import pytest +from contextlib import nullcontext as does_not_raise +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException +from helpers.test_tools import exec_query_with_retry +from helpers.test_tools import assert_eq_with_retry + + +ACCESSIBLE_IPV4 = "10.5.172.10" +OTHER_ACCESSIBLE_IPV4 = "10.5.172.20" +NOT_ACCESSIBLE_IPV4 = "10.5.172.11" + +ACCESSIBLE_IPV6 = "2001:3984:3989::1:1000" +NOT_ACCESSIBLE_IPV6 = "2001:3984:3989::1:1001" + +DST_NODE_IPV4 = ACCESSIBLE_IPV4 +DST_NODE_IPV6 = ACCESSIBLE_IPV6 +SRC_NODE_IPV6 = "2001:3984:3989::1:2000" + + +cluster = ClickHouseCluster(__file__) + +# Destination node +dst_node = cluster.add_instance( + "dst_node", + with_zookeeper=True, + ipv4_address=DST_NODE_IPV4, + ipv6_address=DST_NODE_IPV6, + main_configs=["configs/listen.xml"], +) +# Source node +src_node = cluster.add_instance( + "src_node", + with_zookeeper=True, + ipv6_address=SRC_NODE_IPV6, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + pass + + +@pytest.fixture +def dst_node_addrs(started_cluster, request): + src_node.set_hosts([(ip, "dst_node") for ip in request.param]) + src_node.query("SYSTEM DROP DNS CACHE") + + yield + + # Clear static DNS entries + src_node.set_hosts([]) + src_node.query("SYSTEM DROP DNS CACHE") + + +@pytest.mark.parametrize( + "dst_node_addrs, expectation", + [ + ((ACCESSIBLE_IPV4, ACCESSIBLE_IPV6), does_not_raise()), + ((NOT_ACCESSIBLE_IPV4, ACCESSIBLE_IPV6), does_not_raise()), + ((ACCESSIBLE_IPV4, NOT_ACCESSIBLE_IPV6), does_not_raise()), + ( + (NOT_ACCESSIBLE_IPV4, NOT_ACCESSIBLE_IPV6), + pytest.raises(QueryRuntimeException), + ), + ], + indirect=["dst_node_addrs"], +) +def test_url_destination_host_with_multiple_addrs(dst_node_addrs, expectation): + with expectation: + result = src_node.query( + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + ) + assert result == "42\n" + + +def test_url_invalid_hostname(started_cluster): + with pytest.raises(QueryRuntimeException): + src_node.query( + "SELECT count(*) FROM url('http://notvalidhost:8123/?query=SELECT+1', TSV, 'column1 UInt32');" + ) + + +def test_url_ip_change(started_cluster): + assert ( + src_node.query( + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + ) + == "42\n" + ) + + started_cluster.restart_instance_with_ip_change(dst_node, OTHER_ACCESSIBLE_IPV4) + + # Ensure that only new IPV4 address is accessible + src_node.set_hosts( + [(OTHER_ACCESSIBLE_IPV4, "dst_node"), (NOT_ACCESSIBLE_IPV6, "dst_node")] + ) + src_node.query("SYSTEM DROP DNS CACHE") + + assert ( + src_node.query( + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + ) + == "42\n" + ) From 08defa36b2f9d5a9b9d70a21132d4d766b92c509 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jul 2023 12:09:22 +0300 Subject: [PATCH 210/473] Add code --- .../Net/include/Poco/Net/HTTPClientSession.h | 2 +- .../S3/ProxyResolverConfiguration.cpp | 2 +- src/IO/HTTPCommon.cpp | 134 +++++++++++------- src/IO/HTTPCommon.h | 4 +- src/IO/S3/PocoHTTPClient.cpp | 8 +- 5 files changed, 89 insertions(+), 61 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index d495d662f75..167a06eb7ff 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -306,7 +306,7 @@ namespace Net DEFAULT_KEEP_ALIVE_TIMEOUT = 8 }; - void reconnect(); + virtual void reconnect(); /// Connects the underlying socket to the HTTP server. int write(const char * buffer, std::streamsize length); diff --git a/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp b/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp index 14db39b3f3d..18c644f3680 100644 --- a/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp +++ b/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp @@ -60,7 +60,7 @@ ClientConfigurationPerRequest ProxyResolverConfiguration::getConfiguration(const { auto resolved_endpoint = endpoint; resolved_endpoint.setHost(resolved_hosts[i].toString()); - session = makeHTTPSession(resolved_endpoint, timeouts, false); + session = makeHTTPSession(resolved_endpoint, timeouts); try { diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 2f5e0a172a0..a5816911c09 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -24,9 +25,9 @@ #include +#include #include #include -#include namespace ProfileEvents @@ -54,6 +55,78 @@ namespace session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); } + template + requires std::derived_from + class HTTPSessionAdapter : public Session + { + static_assert(std::has_virtual_destructor_v, "The base class must have a virtual destructor"); + + public: + HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{&Poco::Logger::get("HTTPSessionAdapter")} { } + ~HTTPSessionAdapter() override = default; + + protected: + void reconnect() override + { + // First of all will try to establish connection with last used addr. + if (!Session::getResolvedHost().empty()) + { + try + { + Session::reconnect(); + return; + } + catch (...) + { + Session::close(); + LOG_TRACE( + log, + "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", + Session::getResolvedHost(), + Session::getHost(), + Session::getPort()); + } + } + + const auto endpoinds = DNSResolver::instance().resolveHostAll(Session::getHost()); + + for (auto it = endpoinds.begin();;) + { + try + { + Session::setResolvedHost(it->toString()); + Session::reconnect(); + + LOG_TRACE( + log, + "Created HTTP(S) session with {}:{} ({}:{})", + Session::getHost(), + Session::getPort(), + it->toString(), + Session::getPort()); + + break; + } + catch (...) + { + Session::close(); + if (++it == endpoinds.end()) + { + Session::setResolvedHost(""); + throw; + } + LOG_TRACE( + log, + "Failed to create connection with {}:{}, Will try another resolved address. {}", + Session::getResolvedHost(), + Session::getPort(), + getCurrentExceptionMessage(false)); + } + } + } + Poco::Logger * log; + }; + bool isHTTPS(const Poco::URI & uri) { if (uri.getScheme() == "https") @@ -64,28 +137,21 @@ namespace throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString()); } - HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host = true) + HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive) { HTTPSessionPtr session; if (https) { #if USE_SSL - /// Cannot resolve host in advance, otherwise SNI won't work in Poco. - /// For more information about SNI, see the https://en.wikipedia.org/wiki/Server_Name_Indication - auto https_session = std::make_shared(host, port); - if (resolve_host) - https_session->setResolvedHost(DNSResolver::instance().resolveHost(host).toString()); - - session = std::move(https_session); + session = std::make_shared>(host, port); #else throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without HTTPS support"); #endif } else { - String resolved_host = resolve_host ? DNSResolver::instance().resolveHost(host).toString() : host; - session = std::make_shared(resolved_host, port); + session = std::make_shared>(host, port); } ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); @@ -104,7 +170,6 @@ namespace const String proxy_host; const UInt16 proxy_port; const bool proxy_https; - const bool resolve_host; using Base = PoolBase; @@ -113,7 +178,7 @@ namespace /// Pool is global, we shouldn't attribute this memory to query/user. MemoryTrackerSwitcher switcher{&total_memory_tracker}; - auto session = makeHTTPSessionImpl(host, port, https, true, resolve_host); + auto session = makeHTTPSessionImpl(host, port, https, true); if (!proxy_host.empty()) { const String proxy_scheme = proxy_https ? "https" : "http"; @@ -137,7 +202,6 @@ namespace UInt16 proxy_port_, bool proxy_https_, size_t max_pool_size_, - bool resolve_host_, bool wait_on_pool_size_limit) : Base( static_cast(max_pool_size_), @@ -149,7 +213,6 @@ namespace , proxy_host(proxy_host_) , proxy_port(proxy_port_) , proxy_https(proxy_https_) - , resolve_host(resolve_host_) { } }; @@ -197,24 +260,6 @@ namespace std::mutex mutex; std::unordered_map endpoints_pool; - void updateHostIfIpChanged(Entry & session, const String & new_ip) - { - const auto old_ip = session->getResolvedHost().empty() ? session->getHost() : session->getResolvedHost(); - - if (new_ip != old_ip) - { - session->reset(); - if (session->getResolvedHost().empty()) - { - session->setHost(new_ip); - } - else - { - session->setResolvedHost(new_ip); - } - } - } - protected: HTTPSessionPool() = default; @@ -230,7 +275,6 @@ namespace const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t max_connections_per_endpoint, - bool resolve_host, bool wait_on_pool_size_limit) { std::unique_lock lock(mutex); @@ -261,7 +305,6 @@ namespace proxy_port, proxy_https, max_connections_per_endpoint, - resolve_host, wait_on_pool_size_limit)); /// Some routines held session objects until the end of its lifetime. Also this routines may create another sessions in this time frame. @@ -273,17 +316,6 @@ namespace auto retry_timeout = timeouts.connection_timeout.totalMicroseconds(); auto session = pool_ptr->second->get(retry_timeout); - const auto & session_data = session->sessionData(); - if (session_data.empty() || !Poco::AnyCast(&session_data)) - { - session->reset(); - - if (resolve_host) - updateHostIfIpChanged(session, DNSResolver::instance().resolveHost(host).toString()); - } - - session->attachSessionData({}); - setTimeouts(*session, timeouts); return session; @@ -301,13 +333,13 @@ void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_ response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); } -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host) +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts) { const std::string & host = uri.getHost(); UInt16 port = uri.getPort(); bool https = isHTTPS(uri); - auto session = makeHTTPSessionImpl(host, port, https, false, resolve_host); + auto session = makeHTTPSessionImpl(host, port, https, false); setTimeouts(*session, timeouts); return session; } @@ -317,10 +349,9 @@ PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host, bool wait_on_pool_size_limit) { - return makePooledHTTPSession(uri, {}, timeouts, per_endpoint_pool_size, resolve_host, wait_on_pool_size_limit); + return makePooledHTTPSession(uri, {}, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit); } PooledHTTPSessionPtr makePooledHTTPSession( @@ -328,10 +359,9 @@ PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host, bool wait_on_pool_size_limit) { - return HTTPSessionPool::instance().getSession(uri, proxy_uri, timeouts, per_endpoint_pool_size, resolve_host, wait_on_pool_size_limit); + return HTTPSessionPool::instance().getSession(uri, proxy_uri, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit); } bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 4733f366c8a..082491b2851 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -70,14 +70,13 @@ void markSessionForReuse(PooledHTTPSessionPtr session); void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); /// Create session object to perform requests and set required parameters. -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host = true); +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts); /// As previous method creates session, but tooks it from pool, without and with proxy uri. PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host = true, bool wait_on_pool_size_limit = true); PooledHTTPSessionPtr makePooledHTTPSession( @@ -85,7 +84,6 @@ PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host = true, bool wait_on_pool_size_limit = true); bool isRedirect(Poco::Net::HTTPResponse::HTTPStatus status); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index d64ddf0ec38..1a367a8199d 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -336,9 +336,9 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, /* resolve_host = */ true, wait_on_pool_size_limit); + target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, timeouts, /* resolve_host = */ false); + session = makeHTTPSession(target_uri, timeouts); bool use_tunnel = request_configuration.proxy_scheme == Aws::Http::Scheme::HTTP && target_uri.getScheme() == "https"; session->setProxy( @@ -352,9 +352,9 @@ void PocoHTTPClient::makeRequestInternalImpl( { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, /* resolve_host = */ true, wait_on_pool_size_limit); + target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, timeouts, /* resolve_host = */ false); + session = makeHTTPSession(target_uri, timeouts); } /// In case of error this address will be written to logs From 7d3b28f9c398e4d224243004a6f5c5eddfc4cce2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 21:22:16 +0200 Subject: [PATCH 211/473] Bump version of the distributed_ddl_entry_format_version to 5 by default This will enable the following features for distributed DDL queries: - opentelemetry support (#41484) - initial_query_id pass through (#50015) Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 2 +- ...distributed_ddl_output_mode_long.reference | 32 +++++++++---------- .../02761_ddl_initial_query_id.reference | 2 +- .../0_stateless/02761_ddl_initial_query_id.sh | 5 +-- 4 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..d54645bb0de 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -629,7 +629,7 @@ class IColumn; M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \ M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ - M(UInt64, distributed_ddl_entry_format_version, 3, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ + M(UInt64, distributed_ddl_entry_format_version, 5, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference index 4397810b68d..bc33c2fa913 100644 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference @@ -27,19 +27,19 @@ localhost 9000 57 Code: 57. Error: Table default.never_throw already exists. (TA localhost 9000 0 1 0 localhost 1 \N \N 1 0 distributed_ddl_queue -2 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.throw already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.null_status already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.never_throw already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.throw already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.null_status already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.never_throw already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference index 5c6f448eed5..aac7a0892e1 100644 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference @@ -1,4 +1,4 @@ -default distributed_ddl_entry_format_version +distributed_ddl_entry_format_version=OPENTELEMETRY_ENABLED_VERSION (older then PRESERVE_INITIAL_QUERY_ID_VERSION) DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION DROP TABLE IF EXISTS default.foo diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh index b8b35ef01f7..83890c3f335 100755 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh @@ -4,9 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -echo "default distributed_ddl_entry_format_version" +echo "distributed_ddl_entry_format_version=OPENTELEMETRY_ENABLED_VERSION (older then PRESERVE_INITIAL_QUERY_ID_VERSION)" +OPENTELEMETRY_ENABLED_VERSION=4 query_id="$(random_str 10)" -$CLICKHOUSE_CLIENT --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT --distributed_ddl_entry_format_version=$OPENTELEMETRY_ENABLED_VERSION --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q "SELECT query FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart'" From 7761e5ef9d75bd006e5b0e655682eb0aad9236d8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 15 Jul 2023 14:42:08 +0200 Subject: [PATCH 212/473] support async_deduplicate_token for async insert --- src/Interpreters/AsynchronousInsertQueue.cpp | 14 ++++-- src/Interpreters/AsynchronousInsertQueue.h | 3 +- src/Processors/Chunk.h | 10 ++-- .../MergeTree/MergeTreeDataWriter.cpp | 40 +++++++++------ src/Storages/MergeTree/MergeTreeDataWriter.h | 7 +-- .../MergeTree/ReplicatedMergeTreeSink.cpp | 50 +++++++++++-------- .../MergeTree/tests/gtest_async_inserts.cpp | 31 +++++++----- .../02481_async_insert_dedup.python | 19 +++++-- .../02481_async_insert_dedup_token.reference | 3 ++ .../02481_async_insert_dedup_token.sh | 9 ++++ 10 files changed, 120 insertions(+), 66 deletions(-) create mode 100644 tests/queries/0_stateless/02481_async_insert_dedup_token.reference create mode 100755 tests/queries/0_stateless/02481_async_insert_dedup_token.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index dc2310cfebf..4370c87afba 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -93,6 +93,10 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const for (const auto & setting : settings.allChanged()) { + /// We don't consider this setting because it is only for deduplication, + /// which means we can put two inserts with different tokens in the same block safely. + if (setting.getName() == "insert_deduplication_token") + continue; siphash.update(setting.getName()); applyVisitor(FieldVisitorHash(siphash), setting.getValue()); } @@ -107,9 +111,10 @@ bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) return query_str == other.query_str && settings == other.settings; } -AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_) +AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, const String & async_dedup_token_, MemoryTracker * user_memory_tracker_) : bytes(std::move(bytes_)) , query_id(std::move(query_id_)) + , async_dedup_token(async_dedup_token_) , user_memory_tracker(user_memory_tracker_) , create_time(std::chrono::system_clock::now()) { @@ -222,7 +227,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) /// to avoid buffering of huge amount of data in memory. auto read_buf = getReadBufferFromASTInsertQuery(query); - LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* trow_exception */ false, /* exact_limit */ {}); + LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* throw_exception */ false, /* exact_limit */ {}); WriteBufferFromString write_buf(bytes); copyData(limit_buf, write_buf); @@ -248,7 +253,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) if (auto quota = query_context->getQuota()) quota->used(QuotaType::WRITTEN_BYTES, bytes.size()); - auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), CurrentThread::getUserMemoryTracker()); + auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), settings.insert_deduplication_token, CurrentThread::getUserMemoryTracker()); InsertQuery key{query, settings}; InsertDataPtr data_to_process; @@ -451,7 +456,7 @@ try StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); std::unique_ptr last_buffer; - auto chunk_info = std::make_shared(); + auto chunk_info = std::make_shared(); for (const auto & entry : data->entries) { auto buffer = std::make_unique(entry->bytes); @@ -460,6 +465,7 @@ try size_t num_rows = executor.execute(*buffer); total_rows += num_rows; chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); /// Keep buffer, because it still can be used /// in destructor, while resetting buffer at next iteration. diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index bc60c86d067..a75119988d6 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -92,10 +92,11 @@ private: public: String bytes; const String query_id; + const String async_dedup_token; MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; - Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_); + Entry(String && bytes_, String && query_id_, const String & async_dedup_token, MemoryTracker * user_memory_tracker_); void finish(std::exception_ptr exception_ = nullptr); std::future getFuture() { return promise.get_future(); } diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 413872d512d..7969cc7cceb 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -115,15 +115,17 @@ private: using Chunks = std::vector; /// ChunkOffsets marks offsets of different sub-chunks, which will be used by async inserts. -class ChunkOffsets : public ChunkInfo +class AsyncInsertInfo : public ChunkInfo { public: - ChunkOffsets() = default; - explicit ChunkOffsets(const std::vector & offsets_) : offsets(offsets_) {} + AsyncInsertInfo() = default; + explicit AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) : offsets(offsets_), tokens(tokens_) {} + std::vector offsets; + std::vector tokens; }; -using ChunkOffsetsPtr = std::shared_ptr; +using AsyncInsertInfoPtr = std::shared_ptr; /// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. class ChunkMissingValues : public ChunkInfo diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7e306880e9c..ea5d64212f5 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -171,23 +171,23 @@ void MergeTreeDataWriter::TemporaryPart::finalize() projection->getDataPartStorage().precommitTransaction(); } -std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num) +std::vector scatterAsyncInsertInfoBySelector(AsyncInsertInfoPtr async_insert_info, const IColumn::Selector & selector, size_t partition_num) { - if (nullptr == chunk_offsets) + if (nullptr == async_insert_info) { return {}; } if (selector.empty()) { - return {chunk_offsets}; + return {async_insert_info}; } - std::vector result(partition_num); + std::vector result(partition_num); std::vector last_row_for_partition(partition_num, -1); size_t offset_idx = 0; for (size_t i = 0; i < selector.size(); ++i) { ++last_row_for_partition[selector[i]]; - if (i + 1 == chunk_offsets->offsets[offset_idx]) + if (i + 1 == async_insert_info->offsets[offset_idx]) { for (size_t part_id = 0; part_id < last_row_for_partition.size(); ++part_id) { @@ -196,9 +196,12 @@ std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offs continue; size_t offset = static_cast(last_row + 1); if (result[part_id] == nullptr) - result[part_id] = std::make_shared(); + result[part_id] = std::make_shared(); if (result[part_id]->offsets.empty() || offset > *result[part_id]->offsets.rbegin()) + { result[part_id]->offsets.push_back(offset); + result[part_id]->tokens.push_back(async_insert_info->tokens[offset_idx]); + } } ++offset_idx; } @@ -207,7 +210,7 @@ std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offs } BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( - const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, ChunkOffsetsPtr chunk_offsets) + const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr async_insert_info) { BlocksWithPartition result; if (!block || !block.rows()) @@ -218,8 +221,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. { result.emplace_back(Block(block), Row{}); - if (chunk_offsets != nullptr) - result[0].offsets = std::move(chunk_offsets->offsets); + if (async_insert_info != nullptr) + { + result[0].offsets = std::move(async_insert_info->offsets); + result[0].tokens = std::move(async_insert_info->tokens); + } return result; } @@ -236,7 +242,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( IColumn::Selector selector; buildScatterSelector(partition_columns, partition_num_to_first_row, selector, max_parts); - auto chunk_offsets_with_partition = scatterOffsetsBySelector(chunk_offsets, selector, partition_num_to_first_row.size()); + auto async_insert_info_with_partition = scatterAsyncInsertInfoBySelector(async_insert_info, selector, partition_num_to_first_row.size()); size_t partitions_count = partition_num_to_first_row.size(); result.reserve(partitions_count); @@ -255,8 +261,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( /// NOTE: returning a copy of the original block so that calculated partition key columns /// do not interfere with possible calculated primary key columns of the same name. result.emplace_back(Block(block), get_partition(0)); - if (!chunk_offsets_with_partition.empty()) - result[0].offsets = std::move(chunk_offsets_with_partition[0]->offsets); + if (!async_insert_info_with_partition.empty()) + { + result[0].offsets = std::move(async_insert_info_with_partition[0]->offsets); + result[0].tokens = std::move(async_insert_info_with_partition[0]->tokens); + } return result; } @@ -270,8 +279,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( result[i].block.getByPosition(col).column = std::move(scattered[i]); } - for (size_t i = 0; i < chunk_offsets_with_partition.size(); ++i) - result[i].offsets = std::move(chunk_offsets_with_partition[i]->offsets); + for (size_t i = 0; i < async_insert_info_with_partition.size(); ++i) + { + result[i].offsets = std::move(async_insert_info_with_partition[i]->offsets); + result[i].tokens = std::move(async_insert_info_with_partition[i]->tokens); + } return result; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 795453b2afa..d6c980d83d8 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -23,14 +23,15 @@ struct BlockWithPartition Block block; Row partition; std::vector offsets; + std::vector tokens; BlockWithPartition(Block && block_, Row && partition_) : block(block_), partition(std::move(partition_)) { } - BlockWithPartition(Block && block_, Row && partition_, std::vector && offsets_) - : block(block_), partition(std::move(partition_)), offsets(std::move(offsets_)) + BlockWithPartition(Block && block_, Row && partition_, std::vector && offsets_, std::vector && tokens_) + : block(block_), partition(std::move(partition_)), offsets(std::move(offsets_)), tokens(std::move(tokens_)) { } }; @@ -51,7 +52,7 @@ public: * (split rows by partition) * Works deterministically: if same block was passed, function will return same result in same order. */ - static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, ChunkOffsetsPtr chunk_offsets = nullptr); + static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr chunk_offsets = nullptr); /// This structure contains not completely written temporary part. /// Some writes may happen asynchronously, e.g. for blob storages. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4128654a632..72d19dee038 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -78,7 +78,7 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk unmerged_block_with_partition(std::move(unmerged_block_with_partition_)), part_counters(std::move(part_counters_)) { - initBlockIDMap(); + initBlockIDMap(); } void initBlockIDMap() @@ -209,8 +209,8 @@ std::vector testSelfDeduplicate(std::vector data, std::vectorinsert(datum); } Block block({ColumnWithTypeAndName(std::move(column), DataTypePtr(new DataTypeInt64()), "a")}); - - BlockWithPartition block1(std::move(block), Row(), std::move(offsets)); + std::vector tokens(offsets.size()); + BlockWithPartition block1(std::move(block), Row(), std::move(offsets), std::move(tokens)); ProfileEvents::Counters profile_counters; ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition part( &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::nullopt, std::move(profile_counters)); @@ -242,22 +242,29 @@ namespace size_t start = 0; auto cols = block.block.getColumns(); std::vector block_id_vec; - for (auto offset : block.offsets) + for (size_t i = 0; i < block.offsets.size(); ++i) { - SipHash hash; - for (size_t i = start; i < offset; ++i) + size_t offset = block.offsets[i]; + std::string_view token = block.tokens[i]; + if (token.empty()) { - for (const auto & col : cols) - col->updateHashWithValue(i, hash); - } - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); + SipHash hash; + for (size_t j = start; j < offset; ++j) + { + for (const auto & col : cols) + col->updateHashWithValue(j, hash); + } + union + { + char bytes[16]; + UInt64 words[2]; + } hash_value; + hash.get128(hash_value.bytes); - block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + } + else + block_id_vec.push_back(partition_id + "_" + std::string(token)); start = offset; } @@ -418,18 +425,18 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) convertDynamicColumnsToTuples(block, storage_snapshot); - ChunkOffsetsPtr chunk_offsets; + AsyncInsertInfoPtr async_insert_info; if constexpr (async_insert) { const auto & chunk_info = chunk.getChunkInfo(); - if (const auto * chunk_offsets_ptr = typeid_cast(chunk_info.get())) - chunk_offsets = std::make_shared(chunk_offsets_ptr->offsets); + if (const auto * async_insert_info_ptr = typeid_cast(chunk_info.get())) + async_insert_info = std::make_shared(async_insert_info_ptr->offsets, async_insert_info_ptr->tokens); else throw Exception(ErrorCodes::LOGICAL_ERROR, "No chunk info for async inserts"); } - auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, chunk_offsets); + auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, async_insert_info); using DelayedPartition = typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition; using DelayedPartitions = std::vector; @@ -453,7 +460,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { /// we copy everything but offsets which we move because they are only used by async insert if (settings.optimize_on_insert && storage.writer.getMergingMode() != MergeTreeData::MergingParams::Mode::Ordinary) - unmerged_block.emplace(Block(current_block.block), Row(current_block.partition), std::move(current_block.offsets)); + unmerged_block.emplace(Block(current_block.block), Row(current_block.partition), std::move(current_block.offsets), std::move(current_block.tokens)); } /// Write part to the filesystem under temporary name. Calculate a checksum. @@ -468,7 +475,6 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { - /// TODO consider insert_deduplication_token block_id = getHashesForBlocks(unmerged_block.has_value() ? *unmerged_block : current_block, temp_part.part->info.partition_id); LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}, size {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets), current_block.offsets.size()); } diff --git a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp index f67c2f7fb0f..2d8cd0acc3e 100644 --- a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp +++ b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp @@ -8,7 +8,7 @@ namespace DB { -std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num); +std::vector scatterAsyncInsertInfoBySelector(AsyncInsertInfoPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num); class AsyncInsertsTest : public ::testing::TestPartResult {}; @@ -16,31 +16,36 @@ class AsyncInsertsTest : public ::testing::TestPartResult TEST(AsyncInsertsTest, testScatterOffsetsBySelector) { - auto test_impl = [](std::vector offsets, std::vector selector_data, size_t part_num, std::vector> expected) + auto test_impl = [](std::vector offsets, std::vector selector_data, std::vector tokens, size_t part_num, std::vector>> expected) { - auto offset_ptr = std::make_shared(offsets); + auto offset_ptr = std::make_shared(offsets, tokens); IColumn::Selector selector(selector_data.size()); size_t num_rows = selector_data.size(); for (size_t i = 0; i < num_rows; i++) selector[i] = selector_data[i]; - auto results = scatterOffsetsBySelector(offset_ptr, selector, part_num); + auto results = scatterAsyncInsertInfoBySelector(offset_ptr, selector, part_num); ASSERT_EQ(results.size(), expected.size()); for (size_t i = 0; i < results.size(); i++) { - auto result = results[i]->offsets; + auto result = results[i]; auto expect = expected[i]; - ASSERT_EQ(result.size(), expect.size()); - for (size_t j = 0; j < result.size(); j++) - ASSERT_EQ(result[j], expect[j]); + ASSERT_EQ(result->offsets.size(), expect.size()); + ASSERT_EQ(result->tokens.size(), expect.size()); + for (size_t j = 0; j < expect.size(); j++) + { + ASSERT_EQ(result->offsets[j], std::get<0>(expect[j])); + ASSERT_EQ(result->tokens[j], std::get<1>(expect[j])); + } } }; - test_impl({5}, {0,1,0,1,0}, 2, {{3},{2}}); - test_impl({5,10}, {0,1,0,1,0,1,0,1,0,1}, 2, {{3,5},{2,5}}); - test_impl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, 3, {{2,4},{2,4},{2,4}}); - test_impl({1,2,3,4,5}, {0,1,2,3,4}, 5, {{1},{1},{1},{1},{1}}); - test_impl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, 3, {{4},{3},{3}}); + test_impl({1}, {0}, {"a"}, 1, {{{1,"a"}}}); + test_impl({5}, {0,1,0,1,0}, {"a"}, 2, {{{3,"a"}},{{2,"a"}}}); + test_impl({5,10}, {0,1,0,1,0,1,0,1,0,1}, {"a", "b"}, 2, {{{3,"a"},{5,"b"}},{{2,"a"},{5,"b"}}}); + test_impl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, {"a", "b", "c"}, 3, {{{2, "a"},{4, "b"}},{{2,"a"},{4,"c"}},{{2,"b"},{4,"c"}}}); + test_impl({1,2,3,4,5}, {0,1,2,3,4}, {"a", "b", "c", "d", "e"}, 5, {{{1,"a"}},{{1,"b"}},{{1, "c"}},{{1, "d"}},{{1, "e"}}}); + test_impl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, {"a", "b", "c"}, 3, {{{4, "c"}},{{3, "a"}},{{3, "b"}}}); } std::vector testSelfDeduplicate(std::vector data, std::vector offsets, std::vector hashes); diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 9fd82da1038..ca83253eaf8 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -39,7 +39,7 @@ client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part SYNC") # generate data and push to queue -def generate_data(q, total_number): +def generate_data(q, total_number, use_token): old_data = [] max_chunk_size = 30 partitions = ["2022-11-11 10:10:10", "2022-12-12 10:10:10"] @@ -63,12 +63,16 @@ def generate_data(q, total_number): end = start + chunk_size if end > total_number: end = total_number + + token = "" for i in range(start, end + 1): partition = partitions[random.randint(0, 1)] insert_stmt += "('{}', {}),".format(partition, i) + if use_token: + token = str(i) insert_stmt = insert_stmt[:-1] - q.put(insert_stmt) - old_data.append(insert_stmt) + q.put((insert_stmt, token)) + old_data.append((insert_stmt, token)) last_number = end if end >= total_number: break @@ -80,13 +84,14 @@ def fetch_and_insert_data(q, client): while True: insert = q.get() client.query( - insert, + insert[0], settings={ "async_insert": 1, "async_insert_deduplicate": 1, "wait_for_async_insert": 0, "async_insert_busy_timeout_ms": 1500, "insert_keeper_fault_injection_probability": 0, + "insert_deduplication_token": insert[1], }, ) q.task_done() @@ -110,7 +115,11 @@ ORDER BY (KeyID, EventDate) SETTINGS use_async_block_ids_cache = 1 q = queue.Queue(100) total_number = 10000 -gen = Thread(target=generate_data, args=[q, total_number]) +use_token = False +if sys.argv[-1] == "token": + use_token = True + +gen = Thread(target=generate_data, args=[q, total_number, use_token]) gen.start() for i in range(3): diff --git a/tests/queries/0_stateless/02481_async_insert_dedup_token.reference b/tests/queries/0_stateless/02481_async_insert_dedup_token.reference new file mode 100644 index 00000000000..a91c59a7fc5 --- /dev/null +++ b/tests/queries/0_stateless/02481_async_insert_dedup_token.reference @@ -0,0 +1,3 @@ +5 + +10000 diff --git a/tests/queries/0_stateless/02481_async_insert_dedup_token.sh b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh new file mode 100755 index 00000000000..8ef6eecda24 --- /dev/null +++ b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: long, zookeeper, no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02481_async_insert_dedup.python token From 290c145acfc5accb2344b63356904af5f0a784ba Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 15 Jul 2023 14:53:46 +0000 Subject: [PATCH 213/473] Add array_agg as alias of groupArray for PostgreSQL compatibility --- .../aggregate-functions/reference/grouparray.md | 2 ++ src/AggregateFunctions/AggregateFunctionGroupArray.cpp | 1 + tests/queries/0_stateless/02813_array_agg.reference | 6 ++++++ tests/queries/0_stateless/02813_array_agg.sql | 10 ++++++++++ 4 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02813_array_agg.reference create mode 100644 tests/queries/0_stateless/02813_array_agg.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/grouparray.md b/docs/en/sql-reference/aggregate-functions/reference/grouparray.md index 18048fa4f71..ad678443df6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/grouparray.md +++ b/docs/en/sql-reference/aggregate-functions/reference/grouparray.md @@ -44,3 +44,5 @@ Result: ``` The groupArray function will remove ᴺᵁᴸᴸ value based on the above results. + +- Alias: `array_agg`. diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index bb1368b9ff8..1c54038929b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -125,6 +125,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); + factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/tests/queries/0_stateless/02813_array_agg.reference b/tests/queries/0_stateless/02813_array_agg.reference new file mode 100644 index 00000000000..202f6aa4bac --- /dev/null +++ b/tests/queries/0_stateless/02813_array_agg.reference @@ -0,0 +1,6 @@ +['hello, world!','hello, world!','hello, world!','hello, world!','hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] diff --git a/tests/queries/0_stateless/02813_array_agg.sql b/tests/queries/0_stateless/02813_array_agg.sql new file mode 100644 index 00000000000..91d8d0774d3 --- /dev/null +++ b/tests/queries/0_stateless/02813_array_agg.sql @@ -0,0 +1,10 @@ +drop table if exists t; +create table t (n Int32, s String) engine=MergeTree order by n; + +insert into t select number, 'hello, world!' from numbers (5); + +select array_agg(s) from t; + +select aRray_Agg(s) from t group by n; + +drop table t; From 0d48dca171c146871996046855dfa9790aa421e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 20:36:16 +0300 Subject: [PATCH 214/473] Update materialized_with_ddl.py --- .../test_materialized_mysql_database/materialized_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 73f6e11d7f8..8926c0e5c81 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -992,7 +992,7 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SHOW TABLES FROM db FORMAT TSV", "t\n") clickhouse_node.query("SYSTEM STOP MERGES db.t") - clickhouse_node.query("DROP VIEW v IF EXISTS") + clickhouse_node.query("DROP VIEW IF EXISTS v") clickhouse_node.query("CREATE VIEW v AS SELECT * FROM db.t") mysql_node.query("INSERT INTO db.t VALUES (1, 1), (2, 2)") mysql_node.query("DELETE FROM db.t WHERE a = 2;") From 20b77e946a03605907320d13dbfacfb0f413f828 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:00:23 +0200 Subject: [PATCH 215/473] There is no point in detecting flaky tests --- tests/ci/clickhouse_helper.py | 24 ------------------- tests/ci/compatibility_check.py | 2 -- tests/ci/fast_test_check.py | 2 -- tests/ci/functional_test_check.py | 2 -- tests/ci/install_check.py | 2 -- tests/ci/integration_test_check.py | 2 -- tests/ci/report.py | 2 +- tests/ci/stress_check.py | 2 -- tests/ci/style_check.py | 2 -- tests/ci/unit_tests_check.py | 2 -- tests/integration/ci-runner.py | 38 ++++++++---------------------- 11 files changed, 11 insertions(+), 69 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 64b64896f66..9410b37d69f 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -190,27 +190,3 @@ def prepare_tests_results_for_clickhouse( result.append(current_row) return result - - -def mark_flaky_tests( - clickhouse_helper: ClickHouseHelper, check_name: str, test_results: TestResults -) -> None: - try: - query = f"""SELECT DISTINCT test_name -FROM checks -WHERE - check_start_time BETWEEN now() - INTERVAL 3 DAY AND now() - AND check_name = '{check_name}' - AND (test_status = 'FAIL' OR test_status = 'FLAKY') - AND pull_request_number = 0 -""" - - tests_data = clickhouse_helper.select_json_each_row("default", query) - master_failed_tests = {row["test_name"] for row in tests_data} - logging.info("Found flaky tests: %s", ", ".join(master_failed_tests)) - - for test_result in test_results: - if test_result.status == "FAIL" and test_result.name in master_failed_tests: - test_result.status = "FLAKY" - except Exception as ex: - logging.error("Exception happened during flaky tests fetch %s", ex) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 04203617dca..97de7fed2d5 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -13,7 +13,6 @@ from github import Github from build_download_helper import download_builds_filter from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status @@ -231,7 +230,6 @@ def main(): ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index d5198e5c3d7..460e17acd37 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -14,7 +14,6 @@ from github import Github from build_check import get_release_or_pr from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -190,7 +189,6 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, NAME, test_results) s3_path_prefix = os.path.join( get_release_or_pr(pr_info, get_version_from_repo())[0], pr_info.sha, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 9279b19b187..b773d1eddd9 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -16,7 +16,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -368,7 +367,6 @@ def main(): state = override_status(state, check_name, invert=validate_bugfix_check) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index d619ce96cee..73e1a6ef739 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -15,7 +15,6 @@ from github import Github from build_download_helper import download_builds_filter from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -345,7 +344,6 @@ def main(): return ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) description = format_description(description) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index e6b2203fb65..222b2197117 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -15,7 +15,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -276,7 +275,6 @@ def main(): state = override_status(state, check_name, invert=validate_bugfix_check) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) s3_helper = S3Helper() report_url = upload_results( diff --git a/tests/ci/report.py b/tests/ci/report.py index a9014acec12..8b301d08d56 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -349,7 +349,7 @@ def create_test_html_report( has_log_urls = True row = "" - has_error = test_result.status in ("FAIL", "FLAKY", "NOT_FAILED") + has_error = test_result.status in ("FAIL", "NOT_FAILED") if has_error and test_result.raw_logs is not None: row = '' row += "" + test_result.name + "" diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index ac280916a2f..895eb318bc4 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -13,7 +13,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status @@ -168,7 +167,6 @@ def run_stress_test(docker_image_name): result_path, server_log_path, run_log_path ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 33a5cd21f39..0871dd7ec6a 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -12,7 +12,6 @@ from typing import List, Tuple from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -189,7 +188,6 @@ def main(): state, description, test_results, additional_files = process_result(temp_path) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, NAME, test_results) report_url = upload_results( s3_helper, pr_info.number, pr_info.sha, test_results, additional_files, NAME diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 5279ccde492..1c3ee303b27 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -12,7 +12,6 @@ from github import Github from build_download_helper import download_unit_tests from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -159,7 +158,6 @@ def main(): state, description, test_results, additional_logs = process_results(test_output) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 00bca60e7db..190c79a5263 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -487,36 +487,23 @@ class ClickhouseIntegrationTestsRunner: def _update_counters(self, main_counters, current_counters, broken_tests): for test in current_counters["PASSED"]: - if ( - test not in main_counters["PASSED"] - and test not in main_counters["FLAKY"] - ): - is_flaky = False + if test not in main_counters["PASSED"]: if test in main_counters["FAILED"]: main_counters["FAILED"].remove(test) - is_flaky = True if test in main_counters["ERROR"]: main_counters["ERROR"].remove(test) - is_flaky = True if test in main_counters["BROKEN"]: main_counters["BROKEN"].remove(test) - is_flaky = True - if is_flaky: - main_counters["FLAKY"].append(test) + if test not in broken_tests: + main_counters["PASSED"].append(test) else: - if test not in broken_tests: - main_counters["PASSED"].append(test) - else: - main_counters["NOT_FAILED"].append(test) + main_counters["NOT_FAILED"].append(test) for state in ("ERROR", "FAILED"): for test in current_counters[state]: - if test in main_counters["FLAKY"]: - continue if test in main_counters["PASSED"]: main_counters["PASSED"].remove(test) - main_counters["FLAKY"].append(test) continue if test not in broken_tests: if test not in main_counters[state]: @@ -605,7 +592,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], } tests_times = defaultdict(float) for test in tests_in_group: @@ -627,7 +613,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], "BROKEN": [], "NOT_FAILED": [], } @@ -757,11 +742,11 @@ class ClickhouseIntegrationTestsRunner: ) log_paths.append(extras_result_path) - if len(counters["PASSED"]) + len(counters["FLAKY"]) == len(tests_in_group): + if len(counters["PASSED"]) == len(tests_in_group): logging.info("All tests from group %s passed", test_group) break if ( - len(counters["PASSED"]) + len(counters["FLAKY"]) >= 0 + len(counters["PASSED"]) >= 0 and len(counters["FAILED"]) == 0 and len(counters["ERROR"]) == 0 ): @@ -825,7 +810,7 @@ class ClickhouseIntegrationTestsRunner: result_state = "failure" if not should_fail: break - assert len(counters["FLAKY"]) == 0 or should_fail + assert should_fail logging.info("Try is OK, all tests passed, going to clear env") clear_ip_tables_and_restart_daemons() logging.info("And going to sleep for some time") @@ -835,7 +820,7 @@ class ClickhouseIntegrationTestsRunner: time.sleep(5) test_result = [] - for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"): + for state in ("ERROR", "FAILED", "PASSED", "SKIPPED"): if state == "PASSED": text_state = "OK" elif state == "FAILED": @@ -928,7 +913,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], "BROKEN": [], "NOT_FAILED": [], } @@ -988,7 +972,6 @@ class ClickhouseIntegrationTestsRunner: "FAILED", "PASSED", "SKIPPED", - "FLAKY", "BROKEN", "NOT_FAILED", ): @@ -1004,15 +987,14 @@ class ClickhouseIntegrationTestsRunner: ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}, flaky: {}".format( - failed_sum, len(counters["PASSED"]), len(counters["FLAKY"]) + status_text = "fail: {}, passed: {}".format( + failed_sum, len(counters["PASSED"]) ) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text result_state = "failure" - counters["FLAKY"] = [] if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" From 44fc93ba638e14054e2d17d8ba6e4e8fa578f268 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 15 Jul 2023 21:10:21 +0000 Subject: [PATCH 216/473] Automatic style fix --- tests/integration/ci-runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 190c79a5263..9629a5821b5 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -987,9 +987,7 @@ class ClickhouseIntegrationTestsRunner: ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}".format( - failed_sum, len(counters["PASSED"]) - ) + status_text = "fail: {}, passed: {}".format(failed_sum, len(counters["PASSED"])) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text From 4f6a0c2faccdb0e6652a21467da164b12520846c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:11:22 +0200 Subject: [PATCH 217/473] Loosen the check --- tests/clickhouse-test | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 46ec19b041d..36e793f03e0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1213,17 +1213,20 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - leftover_tables = clickhouse_execute( - args, - f"SHOW TABLES FROM {database}", - timeout=seconds_left, - settings={ - "log_comment": args.testcase_basename, - }, - ).decode().replace("\n", ", "); + # Check if the test does not cleanup its tables. + # Only for newly added tests. Please extend this check to the old tests as well. + if self.case_file >= '02800': + leftover_tables = clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ).decode().replace("\n", ", "); - if 0 != len(leftover_tables): - raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + if 0 != len(leftover_tables): + raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: From 095624fbd17ee7c4b1a68525c59a3d5c14ba4baf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:33:11 +0200 Subject: [PATCH 218/473] Remove default argument value --- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageMergeTree.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 29880f10f28..074f01e7d03 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -373,7 +373,7 @@ void StorageMergeTree::alter( /// Always execute required mutations synchronously, because alters /// should be executed in sequential order. if (!maybe_mutation_commands.empty()) - waitForMutation(mutation_version); + waitForMutation(mutation_version, false); } { @@ -601,7 +601,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer Int64 version = startMutation(commands, query_context); if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) - waitForMutation(version); + waitForMutation(version, false); } bool StorageMergeTree::hasLightweightDeletedMask() const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8099f9c16aa..6aecde15117 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -191,7 +191,7 @@ private: /// and into in-memory structures. Wake up merge-mutation task. Int64 startMutation(const MutationCommands & commands, ContextPtr query_context); /// Wait until mutation with version will finish mutation for all parts - void waitForMutation(Int64 version, bool wait_for_another_mutation = false); + void waitForMutation(Int64 version, bool wait_for_another_mutation); void waitForMutation(const String & mutation_id, bool wait_for_another_mutation) override; void waitForMutation(Int64 version, const String & mutation_id, bool wait_for_another_mutation = false); void setMutationCSN(const String & mutation_id, CSN csn) override; From e76c9612fbaa7ae17dbdf35ca45dea15e7a89bce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 00:26:42 +0200 Subject: [PATCH 219/473] Fix the "kill_mutation" test --- tests/queries/0_stateless/00834_kill_mutation.sh | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation.sh b/tests/queries/0_stateless/00834_kill_mutation.sh index 46b10ccbff4..4c6455d2f53 100755 --- a/tests/queries/0_stateless/00834_kill_mutation.sh +++ b/tests/queries/0_stateless/00834_kill_mutation.sh @@ -27,8 +27,22 @@ ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id FROM system.mutations WHERE dat ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that blocks another mutation ***'" +# Note: there is a benign race condition. +# The mutation can fail with the message +# "Cannot parse string 'a' as UInt32" +# or +# "Cannot parse string 'b' as UInt32" +# depending on which parts are processed first. +# The mutations are also coalesced together, and the subsequent mutation inherits the failure status of the original mutation. +# When we are waiting for mutations, we are listing all the mutations with identical error messages. +# But due to a race condition and to repeated runs, the original and subsequent mutations can have different error messages, +# therefore the original mutation will not be included in the list. + +# Originally, there was grep "happened during execution of mutations 'mutation_4.txt, mutation_5.txt'", +# but due to this race condition, I've replaced it to grep "happened during execution of mutation" + ${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation DELETE WHERE toUInt32(s) = 1" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation DELETE WHERE x = 1 SETTINGS mutations_sync = 1" 2>&1 | grep -o "happened during execution of mutations 'mutation_4.txt, mutation_5.txt'" | head -n 1 +${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation DELETE WHERE x = 1 SETTINGS mutations_sync = 1" 2>&1 | grep -o "happened during execution of mutation" | head -n 1 # but exception doesn't stop mutations, and we will still see them in system.mutations ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'" # 1 From 66b66db39d8d511fe3df137a14586072bf538481 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 02:52:06 +0200 Subject: [PATCH 220/473] Fix ORDER BY tuple of WINDOW functions --- src/Interpreters/ExpressionAnalyzer.cpp | 11 +++++----- src/Interpreters/GetAggregatesVisitor.cpp | 20 +++++++++++++------ src/Interpreters/GetAggregatesVisitor.h | 2 +- src/Interpreters/TreeRewriter.cpp | 1 + src/Interpreters/TreeRewriter.h | 5 ++--- ...4_order_by_tuple_window_function.reference | 1 + .../02814_order_by_tuple_window_function.sql | 1 + 7 files changed, 25 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02814_order_by_tuple_window_function.reference create mode 100644 tests/queries/0_stateless/02814_order_by_tuple_window_function.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f27d23e8e94..9a450fabd5b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1378,10 +1378,9 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool /* only_types */) { ExpressionActionsChain::Step & step = chain.lastStep(columns_after_window); + for (const auto & expression : syntax->expressions_with_window_function) - { getRootActionsForWindowFunctions(expression->clone(), true, step.actions()); - } } void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) @@ -1760,9 +1759,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// second_stage: Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing. /** First we compose a chain of actions and remember the necessary steps from it. - * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and - * throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries. - */ + * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and + * throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries. + */ const ASTSelectQuery & query = *query_analyzer.getSelectQuery(); auto context = query_analyzer.getContext(); @@ -1805,7 +1804,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { - // we evaluate sampling for Merge lazily so we need to get all the columns + // we evaluate sampling for Merge lazily, so we need to get all the columns if (storage->getName() == "Merge") { const auto columns = metadata_snapshot->getColumns().getAll(); diff --git a/src/Interpreters/GetAggregatesVisitor.cpp b/src/Interpreters/GetAggregatesVisitor.cpp index dd958693d89..718721308b1 100644 --- a/src/Interpreters/GetAggregatesVisitor.cpp +++ b/src/Interpreters/GetAggregatesVisitor.cpp @@ -1,4 +1,6 @@ #include +#include + namespace DB { @@ -13,7 +15,7 @@ struct WindowExpressionsCollectorChildInfo bool window_function_in_subtree = false; }; -// This visitor travers AST and collects the list of expressions which depend on +// This visitor traverses the AST and collects the list of expressions which depend on // evaluation of window functions. Expression is collected only if // it's not a part of another expression. // @@ -26,15 +28,18 @@ struct WindowExpressionsCollectorMatcher { if (child->as() || child->as()) return false; + if (auto * select = node->as()) { - // We don't analysis WITH statement because it might contain useless aggregates + // We don't analyse the WITH statement because it might contain useless aggregates if (child == select->with()) return false; } - // We procces every expression manually + + // We process every expression manually if (auto * func = node->as()) return false; + return true; } @@ -50,6 +55,8 @@ struct WindowExpressionsCollectorMatcher ASTPtr & ast, const ASTPtr & parent) { + checkStackSize(); + if (auto * func = ast->as()) { if (func->is_window_function) @@ -67,7 +74,7 @@ struct WindowExpressionsCollectorMatcher { func->compute_after_window_functions = true; if ((!parent || !parent->as())) - expressions_with_window_functions.push_back(func); + expressions_with_window_functions.push_back(ast); } return result; @@ -75,15 +82,16 @@ struct WindowExpressionsCollectorMatcher return {}; } - std::vector expressions_with_window_functions {}; + ASTs expressions_with_window_functions; }; using WindowExpressionsCollectorVisitor = InDepthNodeVisitorWithChildInfo; -std::vector getExpressionsWithWindowFunctions(ASTPtr & ast) +ASTs getExpressionsWithWindowFunctions(ASTPtr & ast) { WindowExpressionsCollectorVisitor visitor; visitor.visit(ast); + return std::move(visitor.expressions_with_window_functions); } diff --git a/src/Interpreters/GetAggregatesVisitor.h b/src/Interpreters/GetAggregatesVisitor.h index 3f5804c39a0..fdf54de3e57 100644 --- a/src/Interpreters/GetAggregatesVisitor.h +++ b/src/Interpreters/GetAggregatesVisitor.h @@ -114,6 +114,6 @@ inline void assertNoAggregates(const ASTPtr & ast, const char * description) GetAggregatesVisitor(data).visit(ast); } -std::vector getExpressionsWithWindowFunctions(ASTPtr & ast); +ASTs getExpressionsWithWindowFunctions(ASTPtr & ast); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index cda5ceeb164..65b5d950975 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1287,6 +1287,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( bool is_changed = replaceAliasColumnsInQuery(query, result.storage_snapshot->metadata->getColumns(), result.array_join_result_to_source, getContext(), excluded_nodes); + /// If query is changed, we need to redo some work to correct name resolution. if (is_changed) { diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index b94043b8983..ea16c432d0f 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -40,11 +40,10 @@ struct TreeRewriterResult NameSet expanded_aliases; Aliases aliases; + std::vector aggregates; - std::vector window_function_asts; - - std::vector expressions_with_window_function; + ASTs expressions_with_window_function; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". diff --git a/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference b/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql b/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql new file mode 100644 index 00000000000..8ba54fc11d0 --- /dev/null +++ b/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql @@ -0,0 +1 @@ +SELECT 1 ORDER BY tuple(count() OVER ()); From a0fef7496926e0a64797294d97c488b6eb0cf59e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 04:08:00 +0200 Subject: [PATCH 221/473] Update test reference --- tests/queries/0_stateless/00834_kill_mutation.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation.reference b/tests/queries/0_stateless/00834_kill_mutation.reference index 1685343c2b1..49fabab2f8a 100644 --- a/tests/queries/0_stateless/00834_kill_mutation.reference +++ b/tests/queries/0_stateless/00834_kill_mutation.reference @@ -2,7 +2,7 @@ 1 waiting default kill_mutation mutation_3.txt DELETE WHERE toUInt32(s) = 1 *** Create and kill invalid mutation that blocks another mutation *** -happened during execution of mutations 'mutation_4.txt, mutation_5.txt' +happened during execution of mutation 1 waiting default kill_mutation mutation_4.txt DELETE WHERE toUInt32(s) = 1 2001-01-01 2 b From 4669951db550f4d70039c678ffacafa15b966f6c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 09:06:29 +0200 Subject: [PATCH 222/473] Fix timeout for hedged requests Reset the timeout for hedged requests after each packet (such as Progress packets) to avoid triggering send_timeout/receive_timeout as a total timeout for the query in some cases. Here is a simple query that demonstrates the problem: select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1 Signed-off-by: Azat Khuzhin --- src/Client/HedgedConnections.cpp | 2 ++ .../0_stateless/02805_distributed_queries_timeouts.reference | 0 .../queries/0_stateless/02805_distributed_queries_timeouts.sql | 3 +++ 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.reference create mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.sql diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fb6fe258d48..0efad1188fa 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -353,6 +353,8 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc if (replica_state.packet_receiver->isPacketReady()) { + /// Reset the socket timeout after some packet received + replica_state.packet_receiver->setTimeout(hedged_connections_factory.getConnectionTimeouts().receive_timeout); last_received_packet = replica_state.packet_receiver->getPacket(); return true; } diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference b/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql new file mode 100644 index 00000000000..0b7337d1255 --- /dev/null +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -0,0 +1,3 @@ +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; From 20a671b8cf2830733eea2ed7e7c6cd80d70ed81a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 15 Jul 2023 19:44:29 +0200 Subject: [PATCH 223/473] Skip protection from double decompression if inode from maps cannot be obtained Under some circumstances, like using qemu-$ARCH-static, /proc/self/maps will not contain information about /proc/self/exe. Well, strictly speaking it does contains, however qemu will not pass it to the user program:
strace $ sudo strace -s10000 -f arch-chroot . /qemu-riscv64-static /clickhouse ... execve("/qemu-riscv64-static", ["/qemu-riscv64-static", "/clickhouse"], 0x7fffffffe458 /* 20 vars */) = 0 readlinkat(AT_FDCWD, "/proc/self/exe", "/qemu-riscv64-static", 4096) = 20 openat(AT_FDCWD, "/proc/self/maps", O_RDONLY|O_CLOEXEC) = 4 [pid 3126] read(4, "00010000-00111000 r--p 00000000 fe:01 30312571 /clickhouse\n00111000-00119000 r--p 00100000 fe:01 30312571 /clickhouse\n00119000-0011a000 rw-p 00108000 fe:01 30312571 /clickhouse\n0011a000-0013d000 rw-p 00000000 00:00 0 \n4000000000-4000001000 ---p 00000000 00:00 0 \n4000001000-4000801000 rw-p 00000000 00:00 0 \n4000801000-400081a000 r--p 00000000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081a000-400081b000 ---p 00000000 00:00 0 \n400081b000-400081c000 r--p 00019000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081c000-400081e000 rw-p 0001a000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081e000-400081f000 r--p 00000000 00:00 0 \n400081f000-4000922000 r--p 00000000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000922000-4000926000 r--p 00102000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000926000-4000928000 rw-p 00106000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000928000-400092d000 rw-p 00000000 00:00 0 \n400092d000-40009af000 r--p 00000000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009af000-40009b0000 r--p 00081000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009b0000-40009b1000 rw-p 00082000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009b1000-40009c5000 r--p 00000000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c5000-40009c6000 r--p 00013000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c6000-40009c7000 rw-p 00014000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c7000-40009cb000 rw-p 00000000 00:00 0 \n40009cb000-40009cd000 r--p 00000000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009cd000-40009ce000 r--p 00001000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009ce000-40009cf000 rw-p 00002000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009cf000-40009d1000 rw-p 00000000 00:00 0 \n7fffe8000000-7fffeffff000 rwxp 00000000 00:00 0 \n7fffeffff000-7ffff0000000 ---p 00000000 00:00 0 \n7ffff0000000-7ffff0021000 rw-p 00000000 00:00 0 \n7ffff0021000-7ffff4000000 ---p 00000000 00:00 0 \n7ffff6b4b000-7ffff6b5b000 rw-p 00000000 00:00 0 \n7ffff71ff000-7ffff7200000 ---p 00000000 00:00 0 \n7ffff7200000-7ffff7a00000 rw-p 00000000 00:00 0\n7ffff7a00000-7ffff7a3c000 r--p 00000000 fe:01 30316953 /qemu-riscv64-static\n7ffff7a3c000-7ffff7c74000 r-xp 0003c000 fe:01 30316953 /qemu-riscv64-static\n7ffff7c74000-7ffff7d77000 r--p 00274000 fe:01 30316953 /qemu-riscv64-static\n7ffff7d77000-7ffff7dce000 r--p 00377000 fe:01 30316953 /qemu-riscv64-static\n7ffff7dce000-7ffff7df7000 rw-p 003ce000 fe:01 30316953 /qemu-riscv64-static\n7ffff7df7000-7ffff7e0c000 rw-p 00000000 00:00 0 [heap]\n7ffff7e0c000-7ffff7e70000 rw-p 00000000 00:00 0 [heap]\n7ffff7f42000-7ffff7ff9000 rw-p 00000000 00:00 0 \n7ffff7ff9000-7ffff7ffd000 r--p 00000000 00:00 0 [vvar]\n7ffff7ffd000-7ffff7fff000 r-xp 00000000 00:00 0 [vdso]\n7ffffffde000-7ffffffff000 rw-p 00000000 00:00 0 [stack]\nffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall]\n", 4096) = 3608 [pid 3126] read(4, "", 1024) = 0 [pid 3126] close(4) = 0 [pid 3126] write(3, "10000-111000 r-xp 00000000 fe:01 30312571", 41) = 41 [pid 3126] write(3, " /clickhouse\n", 44) = 44 [pid 3126] write(3, "111000-119000 r--p 00100000 fe:01 30312571", 42) = 42 [pid 3126] write(3, " /clickhouse\n", 43) = 43 [pid 3126] write(3, "119000-11a000 rw-p 00108000 fe:01 30312571", 42) = 42 [pid 3126] write(3, " /clickhouse\n", 43) = 43 [pid 3126] write(3, "11a000-13d000 rw-p 00000000 00:00 0", 35) = 35 [pid 3126] write(3, " \n", 39) = 39 [pid 3126] write(3, "4000000000-4000001000 ---p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "4000001000-4000801000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " [stack]\n", 38) = 38 [pid 3126] write(3, "4000801000-400081a000 r-xp 00000000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57 [pid 3127] <... clock_nanosleep resumed>0x7ffff79ff060) = 0 [pid 3126] <... write resumed>) = 57 [pid 3127] clock_nanosleep(CLOCK_REALTIME, 0, {tv_sec=0, tv_nsec=10000000}, [pid 3126] write(3, "400081a000-400081b000 ---p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400081b000-400081c000 r--p 00019000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57) = 57 [pid 3126] write(3, "400081c000-400081e000 rw-p 0001a000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57) = 57 [pid 3126] write(3, "400081e000-400081f000 r-xp 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400081f000-4000922000 r-xp 00000000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000922000-4000926000 r--p 00102000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000926000-4000928000 rw-p 00106000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000928000-400092d000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400092d000-40009af000 r-xp 00000000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009af000-40009b0000 r--p 00081000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009b0000-40009b1000 rw-p 00082000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009b1000-40009c5000 r-xp 00000000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c5000-40009c6000 r--p 00013000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c6000-40009c7000 rw-p 00014000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c7000-40009cb000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "40009cb000-40009cd000 r-xp 00000000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009cd000-40009ce000 r--p 00001000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009ce000-40009cf000 rw-p 00002000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009cf000-40009d1000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31
Signed-off-by: Azat Khuzhin --- .../decompressor.cpp | 93 ++++++++++--------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index d41b9b1ebe1..4a4985120fd 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -430,55 +430,58 @@ int main(int/* argc*/, char* argv[]) return 1; } + int lock = -1; + /// Protection from double decompression #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); - if (inode == 0) + /// In some cases /proc/self/maps may not contain the inode for the + /// /proc/self/exe, one of such examples are using qemu-*-static, in this + /// case maps will be proxied through the qemu, and it will remove + /// information about itself from it. + if (inode != 0) { - std::cerr << "Unable to obtain inode." << std::endl; - return 1; - } - - std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; - int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); - if (lock < 0) - { - perror("lock open"); - return 1; - } - - /// lock file should be closed on exec call - fcntl(lock, F_SETFD, FD_CLOEXEC); - - if (lockf(lock, F_LOCK, 0)) - { - perror("lockf"); - return 1; - } - - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to - /// 32bit conversion of input_info.st_ino - if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) - input_info.st_ino &= 0x00000000FFFFFFFF; - - /// if decompression was performed by another process since this copy was started - /// then file referred by path "self" is already pointing to different inode - if (input_info.st_ino != inode) - { - struct stat lock_info; - if (0 != fstat(lock, &lock_info)) + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) { - perror("fstat lock"); + perror("lock open"); return 1; } - /// size 1 of lock file indicates that another decompressor has found active executable - if (lock_info.st_size == 1) - execv(self, argv); + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); - printf("No target executable - decompression only was performed.\n"); - return 0; + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to + /// 32bit conversion of input_info.st_ino + if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) + input_info.st_ino &= 0x00000000FFFFFFFF; + + /// if decompression was performed by another process since this copy was started + /// then file referred by path "self" is already pointing to different inode + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) + { + perror("fstat lock"); + return 1; + } + + /// size 1 of lock file indicates that another decompressor has found active executable + if (lock_info.st_size == 1) + execv(self, argv); + + printf("No target executable - decompression only was performed.\n"); + return 0; + } } #endif @@ -546,21 +549,19 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { -#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed - write(lock, "1", 1); -#endif + if (lock >= 0) + write(lock, "1", 1); execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } -#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 - ftruncate(lock, 0); -#endif + if (lock >= 0) + ftruncate(lock, 0); printf("No target executable - decompression only was performed.\n"); } From 6c5fa1d4a971aa6f099bdde2f907e89239fa4563 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 17:46:28 +0100 Subject: [PATCH 224/473] Enable PREWHERE splitting into steps --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f304f0aab6..fc49923b221 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -127,8 +127,8 @@ class IColumn; \ M(Bool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \ - M(Bool, move_all_conditions_to_prewhere, false, "Move all viable conditions from WHERE to PREWHERE", 0) \ - M(Bool, enable_multiple_prewhere_read_steps, false, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ + M(Bool, move_all_conditions_to_prewhere, true, "Move all viable conditions from WHERE to PREWHERE", 0) \ + M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ From 420446e1a41bbc3006b866000c8e35a0dcdfed13 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 24 Feb 2023 19:13:12 +0100 Subject: [PATCH 225/473] Update tests --- .../01582_move_to_prewhere_compact_parts.reference | 3 +-- .../01824_move_to_prewhere_many_columns.reference | 6 ++---- .../0_stateless/01917_prewhere_column_type.reference | 1 + tests/queries/0_stateless/01917_prewhere_column_type.sql | 2 +- .../0_stateless/02156_storage_merge_prewhere.reference | 3 +-- 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference b/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference index 6b762abd192..30b5ae9c648 100644 --- a/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference +++ b/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference @@ -9,5 +9,4 @@ SELECT x3, x4 FROM prewhere_move -PREWHERE x1 > 100 -WHERE (x1 > 100) AND ((x2 > 100) AND (x3 > 100) AND (x4 > 100)) +PREWHERE (x1 > 100) AND (x2 > 100) AND (x3 > 100) AND (x4 > 100) diff --git a/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference b/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference index adce19321d5..686a864f222 100644 --- a/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference +++ b/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference @@ -3,12 +3,10 @@ 35 SELECT count() FROM t_move_to_prewhere -PREWHERE a AND b AND c -WHERE (a AND b AND c) AND (NOT ignore(fat_string)) +PREWHERE a AND b AND c AND (NOT ignore(fat_string)) 1 Compact 2 Compact 35 SELECT count() FROM t_move_to_prewhere -PREWHERE a -WHERE a AND (b AND c AND (NOT ignore(fat_string))) +PREWHERE a AND b AND c AND (NOT ignore(fat_string)) diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.reference b/tests/queries/0_stateless/01917_prewhere_column_type.reference index 58c9bdf9d01..2bbe845f4ef 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.reference +++ b/tests/queries/0_stateless/01917_prewhere_column_type.reference @@ -1 +1,2 @@ 111 +111 diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.sql b/tests/queries/0_stateless/01917_prewhere_column_type.sql index c0bc0c3e36b..7ddcb17fbb9 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.sql +++ b/tests/queries/0_stateless/01917_prewhere_column_type.sql @@ -6,7 +6,7 @@ CREATE TABLE t1 ( s String, f Float32, e UInt16 ) ENGINE = MergeTree ORDER BY tu INSERT INTO t1 VALUES ('111', 1, 1); -SELECT s FROM t1 WHERE f AND (e = 1); -- { serverError 59 } +SELECT s FROM t1 WHERE f AND (e = 1); SELECT s FROM t1 PREWHERE f; -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE (e = 1); -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE f AND (e = 1); -- { serverError 59 } diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 2dc83f1eaa5..30f9b1ab175 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,7 +1,6 @@ SELECT count() FROM t_02156_merge1 -PREWHERE k = 3 -WHERE (k = 3) AND notEmpty(v) +PREWHERE (k = 3) AND notEmpty(v) 2 SELECT count() FROM t_02156_merge2 From d23f67bb04ad677f2c03187201fa941956df8be8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 28 Feb 2023 13:06:11 +0100 Subject: [PATCH 226/473] Couple more cases with float --- tests/queries/0_stateless/01917_prewhere_column_type.reference | 3 +++ tests/queries/0_stateless/01917_prewhere_column_type.sql | 3 +++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.reference b/tests/queries/0_stateless/01917_prewhere_column_type.reference index 2bbe845f4ef..99c26d55eda 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.reference +++ b/tests/queries/0_stateless/01917_prewhere_column_type.reference @@ -1,2 +1,5 @@ 111 111 +111 +111 +111 diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.sql b/tests/queries/0_stateless/01917_prewhere_column_type.sql index 7ddcb17fbb9..9ce87ab548c 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.sql +++ b/tests/queries/0_stateless/01917_prewhere_column_type.sql @@ -7,6 +7,9 @@ CREATE TABLE t1 ( s String, f Float32, e UInt16 ) ENGINE = MergeTree ORDER BY tu INSERT INTO t1 VALUES ('111', 1, 1); SELECT s FROM t1 WHERE f AND (e = 1); +SELECT s FROM t1 WHERE f AND (e = 1) SETTINGS optimize_move_to_prewhere=true; +SELECT s FROM t1 WHERE f AND (e = 1) SETTINGS optimize_move_to_prewhere=false; +SELECT s FROM t1 PREWHERE f AND (e = 1); SELECT s FROM t1 PREWHERE f; -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE (e = 1); -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE f AND (e = 1); -- { serverError 59 } From 2f9043f16cfd24bfb6f9a8160dfd6dfa5242f1b1 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 5 Jun 2023 20:11:36 +0200 Subject: [PATCH 227/473] Properly deal with columns computed at previous steps --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 17 ++++++++++++++++- ...teps_in_prewhere_reuse_computation.reference | 14 ++++++++++++++ ...read_steps_in_prewhere_reuse_computation.sql | 17 +++++++++++++++++ 3 files changed, 47 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index c7434eab05d..d830ba37e71 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -328,11 +328,22 @@ MergeTreeReadTaskColumns getReadTaskColumns( NameSet columns_from_previous_steps; auto add_step = [&](const PrewhereExprStep & step) { - Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames(); + Names step_column_names; + + /// Computation results from previous steps might be used in the current step as well. In such a case these + /// computed columns will be present in the current step inputs. They don't need to be read from the disk so + /// exclude them from the list of columns to read. This filtering must be done before injecting required + /// columns to avoid adding unnecessary columns or failing to find required columns that are computation + /// results from previous steps. + /// Example: step1: sin(a)>b, step2: sin(a)>c + for (const auto & name : step.actions->getActionsDAG().getRequiredColumnsNames()) + if (!columns_from_previous_steps.contains(name)) + step_column_names.push_back(name); injectRequiredColumns( data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); + /// More columns could have been added, filter them as well by the list of columns from previous steps. Names columns_to_read_in_step; for (const auto & name : step_column_names) { @@ -343,6 +354,10 @@ MergeTreeReadTaskColumns getReadTaskColumns( columns_from_previous_steps.insert(name); } + /// Add results of the step to the list of already "known" columns so that we don't read or compute them again. + for (const auto & name : step.actions->getActionsDAG().getNames()) + columns_from_previous_steps.insert(name); + result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step)); }; diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference new file mode 100644 index 00000000000..904d46b184a --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference @@ -0,0 +1,14 @@ +-- { echoOn } + +SELECT a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +1 +2 +SELECT sin(a) > 2 FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +0 +0 +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c AND sin(a) > -a; +1 +1 +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND a <= c AND sin(a) > -a; +1 +1 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql new file mode 100644 index 00000000000..544f5f03cad --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t_02559; +CREATE TABLE t_02559 (a Int64, b Int64, c Int64) ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_02559 SELECT number, number, number FROM numbers(3); + +SET enable_multiple_prewhere_read_steps = 1; + +-- { echoOn } + +SELECT a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +SELECT sin(a) > 2 FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c AND sin(a) > -a; +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND a <= c AND sin(a) > -a; + +-- {echoOff} + +DROP TABLE t_02559; From 4ee0bacd945415e9d252d89697541c101e8efaf4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 16:38:54 +0200 Subject: [PATCH 228/473] Fix indentation --- ...771_ignore_data_skipping_indices.reference | 74 +++++++++---------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference index 786360783fd..fcede2caf2a 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -1,40 +1,40 @@ 1 2 3 1 2 3 1 2 3 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - Skip - Name: xy_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 From 3e4182fc4fa7f0852aa5e28acd09eb9156f67caf Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 23:19:43 +0200 Subject: [PATCH 229/473] Include source columns for sampling into required columns list --- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ca8a412bf2e..3352567943a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -138,8 +138,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) { const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); - const auto & sampling_columns = sampling_key.sample_block.getColumnsWithTypeAndName(); - required_columns_after_filter.insert(required_columns_after_filter.end(), sampling_columns.begin(), sampling_columns.end()); + const auto & sampling_source_columns = sampling_key.expression->getRequiredColumnsWithTypes(); + for (const auto & column : sampling_source_columns) + required_columns_after_filter.push_back(ColumnWithTypeAndName(column.type, column.name)); + const auto & sampling_result_columns = sampling_key.sample_block.getColumnsWithTypeAndName(); + required_columns_after_filter.insert(required_columns_after_filter.end(), sampling_result_columns.begin(), sampling_result_columns.end()); } const auto & storage = storage_snapshot->storage; From 68a2c6301f982086bf5bebf47b827212d653bedd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 6 Jul 2023 14:21:42 +0200 Subject: [PATCH 230/473] Fix 01786_explain_merge_tree --- .../01786_explain_merge_tree.reference | 160 +++++++++--------- 1 file changed, 79 insertions(+), 81 deletions(-) diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index 8d3954484dd..794acc310ce 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -1,85 +1,83 @@ - ReadFromMergeTree (default.test_index) - Indexes: - MinMax - Keys: - y - Condition: (y in [1, +Inf)) - Parts: 4/5 - Granules: 11/12 - Partition - Keys: - y - bitAnd(z, 3) - Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) - Parts: 3/4 - Granules: 10/11 - PrimaryKey - Keys: - x - y - Condition: and((x in [11, +Inf)), (y in [1, +Inf))) - Parts: 2/3 - Granules: 6/10 - Skip - Name: t_minmax - Description: minmax GRANULARITY 2 - Parts: 1/2 - Granules: 4/6 - Skip - Name: t_set - Description: set GRANULARITY 2 - Parts: 1/1 - Granules: 2/4 + ReadFromMergeTree (default.test_index) + Indexes: + MinMax + Keys: + y + Condition: (y in [1, +Inf)) + Parts: 4/5 + Granules: 11/12 + Partition + Keys: + y + bitAnd(z, 3) + Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) + Parts: 3/4 + Granules: 10/11 + PrimaryKey + Keys: + x + y + Condition: and((x in [11, +Inf)), (y in [1, +Inf))) + Parts: 2/3 + Granules: 6/10 + Skip + Name: t_minmax + Description: minmax GRANULARITY 2 + Parts: 1/2 + Granules: 4/6 + Skip + Name: t_set + Description: set GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 ----------------- - "Node Type": "ReadFromMergeTree", - "Description": "default.test_index", - "Indexes": [ - { - "Type": "MinMax", - "Keys": ["y"], - "Condition": "(y in [1, +Inf))", - "Initial Parts": 5, - "Selected Parts": 4, - "Initial Granules": 12, - "Selected Granules": 11 - }, - { - "Type": "Partition", - "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", - "Initial Parts": 4, - "Selected Parts": 3, - "Initial Granules": 11, - "Selected Granules": 10 - }, - { - "Type": "PrimaryKey", - "Keys": ["x", "y"], - "Condition": "and((x in [11, +Inf)), (y in [1, +Inf)))", - "Initial Parts": 3, - "Selected Parts": 2, - "Initial Granules": 10, - "Selected Granules": 6 - }, - { - "Type": "Skip", - "Name": "t_minmax", - "Description": "minmax GRANULARITY 2", - "Initial Parts": 2, - "Selected Parts": 1, - "Initial Granules": 6, - "Selected Granules": 4 - }, - { - "Type": "Skip", - "Name": "t_set", - "Description": "set GRANULARITY 2", - "Initial Parts": 1, - "Selected Parts": 1, - "Initial Granules": 4, - "Selected Granules": 2 - } - ] + "Node Type": "ReadFromMergeTree", + "Description": "default.test_index", + "Indexes": [ + { + "Type": "MinMax", + "Keys": ["y"], + "Condition": "(y in [1, +Inf))", + "Initial Parts": 5, + "Selected Parts": 4, + "Initial Granules": 12, + "Selected Granules": 11 + }, + { + "Type": "Partition", + "Keys": ["y", "bitAnd(z, 3)"], + "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", + "Initial Parts": 4, + "Selected Parts": 3, + "Initial Granules": 11, + "Selected Granules": 10 + }, + { + "Type": "PrimaryKey", + "Keys": ["x", "y"], + "Condition": "and((x in [11, +Inf)), (y in [1, +Inf)))", + "Initial Parts": 3, + "Selected Parts": 2, + "Initial Granules": 10, + "Selected Granules": 6 + }, + { + "Type": "Skip", + "Name": "t_minmax", + "Description": "minmax GRANULARITY 2", + "Initial Parts": 2, + "Selected Parts": 1, + "Initial Granules": 6, + "Selected Granules": 4 + }, + { + "Type": "Skip", + "Name": "t_set", + "Description": "set GRANULARITY 2", + "Initial Parts": 1, + "Selected Parts": 1, + "Initial Granules": 4, + "Selected Granules": 2 } ] } From 6886e843deff60f03b93f28802816261ed330e27 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 14 Jul 2023 17:23:46 +0200 Subject: [PATCH 231/473] Fix flakiness due to randomized enable_multiple_prewhere_read_steps setting --- tests/queries/1_stateful/00091_prewhere_two_conditions.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql index 745bb125c2b..cbfbbaa2662 100644 --- a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql +++ b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql @@ -4,6 +4,7 @@ SET max_bytes_to_read = 600000000; SET optimize_move_to_prewhere = 1; +SET enable_multiple_prewhere_read_steps = 1; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; @@ -11,6 +12,8 @@ SELECT uniq(*) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014 WITH toTimeZone(EventTime, 'Asia/Dubai') AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; SET optimize_move_to_prewhere = 0; +SET enable_multiple_prewhere_read_steps = 0; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -- { serverError 307 } SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -- { serverError 307 } +SELECT uniq(URL) FROM test.hits PREWHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -- { serverError 307 } From a4f7b7717768893983be2973cef9b1a5684b1949 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:21:00 +0000 Subject: [PATCH 232/473] Add array_concat_agg for compatibility with BigQuery --- .../AggregateFunctionFactory.cpp | 29 ++++++++++++++++++- .../AggregateFunctionFactory.h | 3 ++ .../registerAggregateFunctions.cpp | 8 +++++ src/Common/IFactoryWithAliases.h | 20 ++++++------- .../02813_array_concat_agg.reference | 5 ++++ .../0_stateless/02813_array_concat_agg.sql | 9 ++++++ 6 files changed, 63 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02813_array_concat_agg.reference create mode 100644 tests/queries/0_stateless/02813_array_concat_agg.sql diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 6cacf66500f..f2dbb931b1f 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -59,6 +59,34 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat } } +void AggregateFunctionFactory::registerAliasForAggregateFunctionWithCombinator( + const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) +{ + if (!isAggregateFunctionName(real_name)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}: can't create alias '{}', the real name '{}' is not registered", + getFactoryName(), + alias_name, + real_name); + + auto alias_name_lowercase = Poco::toLower(alias_name); + + if (aggregate_functions.contains(alias_name) || case_insensitive_aggregate_functions.contains(alias_name_lowercase)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", getFactoryName(), alias_name); + + if (case_sensitiveness == CaseInsensitive) + { + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", getFactoryName(), alias_name); + case_insensitive_name_mapping[alias_name_lowercase] = real_name; + } + + if (!aliases.emplace(alias_name, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", getFactoryName(), alias_name); +} + static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; @@ -222,7 +250,6 @@ AggregateFunctionPtr AggregateFunctionFactory::tryGet( : nullptr; } - std::optional AggregateFunctionFactory::tryGetProperties(String name) const { if (name.size() > MAX_AGGREGATE_FUNCTION_NAME_LENGTH) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index dab0d28e851..6c2b539bd8c 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -62,6 +62,9 @@ public: Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerAliasForAggregateFunctionWithCombinator( + const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); + /// Throws an exception if not found. AggregateFunctionPtr get(const String & name, diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 91248a52ae9..87d96f0d1ce 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -193,6 +193,14 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorDistinct(factory); registerAggregateFunctionCombinatorMap(factory); } + + { + auto & factory = AggregateFunctionFactory::instance(); + + /// Must register after registerCombinator + factory.registerAliasForAggregateFunctionWithCombinator( + "array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); + } } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index b2ac4ab289e..af5656ffb75 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -10,6 +10,8 @@ namespace DB { +class AggregateFunctionFactory; + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -22,6 +24,7 @@ namespace ErrorCodes template class IFactoryWithAliases : public IHints<2, IFactoryWithAliases> { + friend AggregateFunctionFactory; protected: using Value = ValueType; @@ -55,9 +58,9 @@ public: const String factory_name = getFactoryName(); String real_dict_name; - if (creator_map.count(real_name)) + if (creator_map.contains(real_name)) real_dict_name = real_name; - else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.count(real_name_lowercase)) + else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.contains(real_name_lowercase)) real_dict_name = real_name_lowercase; else throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: can't create alias '{}', the real name '{}' is not registered", @@ -65,7 +68,7 @@ public: String alias_name_lowercase = Poco::toLower(alias_name); - if (creator_map.count(alias_name) || case_insensitive_creator_map.count(alias_name_lowercase)) + if (creator_map.contains(alias_name) || case_insensitive_creator_map.contains(alias_name_lowercase)) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", factory_name, alias_name); @@ -93,7 +96,7 @@ public: bool isCaseInsensitive(const String & name) const { String name_lowercase = Poco::toLower(name); - return getCaseInsensitiveMap().count(name_lowercase) || case_insensitive_aliases.count(name_lowercase); + return getCaseInsensitiveMap().contains(name_lowercase) || case_insensitive_aliases.contains(name_lowercase); } const String & aliasTo(const String & name) const @@ -106,14 +109,11 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: name '{}' is not alias", getFactoryName(), name); } - bool isAlias(const String & name) const - { - return aliases.count(name) || case_insensitive_aliases.contains(name); - } + bool isAlias(const String & name) const { return aliases.contains(name) || case_insensitive_aliases.contains(name); } bool hasNameOrAlias(const String & name) const { - return getMap().count(name) || getCaseInsensitiveMap().count(name) || isAlias(name); + return getMap().contains(name) || getCaseInsensitiveMap().contains(name) || isAlias(name); } /// Return the canonical name (the name used in registration) if it's different from `name`. @@ -129,7 +129,7 @@ public: private: using InnerMap = std::unordered_map; // name -> creator - using AliasMap = std::unordered_map; // alias -> original type + using AliasMap = std::unordered_map; // alias -> original name virtual const InnerMap & getMap() const = 0; virtual const InnerMap & getCaseInsensitiveMap() const = 0; diff --git a/tests/queries/0_stateless/02813_array_concat_agg.reference b/tests/queries/0_stateless/02813_array_concat_agg.reference new file mode 100644 index 00000000000..7144a499922 --- /dev/null +++ b/tests/queries/0_stateless/02813_array_concat_agg.reference @@ -0,0 +1,5 @@ +[1,2,3,4,5,6] +[1,2,3,4,5,6] +1 [1,2,3] +2 [4,5] +3 [6] diff --git a/tests/queries/0_stateless/02813_array_concat_agg.sql b/tests/queries/0_stateless/02813_array_concat_agg.sql new file mode 100644 index 00000000000..94fe133db7d --- /dev/null +++ b/tests/queries/0_stateless/02813_array_concat_agg.sql @@ -0,0 +1,9 @@ +drop table if exists t; + +create table t (n UInt32, a Array(Int32)) engine=Memory; +insert into t values (1, [1,2,3]), (2, [4,5]), (3, [6]); + +select array_concat_agg(a) from t; +select ArrAy_cOncAt_aGg(a) from t; +select n, array_concat_agg(a) from t group by n order by n; +drop table t; From 4955e07326c4220af989a8938dda9fc3c783c9de Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:27:34 +0000 Subject: [PATCH 233/473] add docs --- .../reference/arrayconcatagg.md | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md new file mode 100644 index 00000000000..50e4ed63787 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md @@ -0,0 +1,32 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/arrayconcatagg +sidebar_position: 110 +--- + +# array_concat_agg +- Alias of `groupArrayArray`. The function is case insensitive. + +**Exampla** + +```text +SELECT * +FROM t + +┌─a───────┐ +│ [1,2,3] │ +│ [4,5] │ +│ [6] │ +└─────────┘ + +``` + +Query: + +```sql +SELECT array_concat_agg(a) AS a +FROM t + +┌─a─────────────┐ +│ [1,2,3,4,5,6] │ +└───────────────┘ +``` From f59370e5f98f72fa136529ed4a4f545faf18b510 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:50:29 +0000 Subject: [PATCH 234/473] Fix style --- .../reference/{arrayconcatagg.md => array_concat_agg.md} | 2 +- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename docs/en/sql-reference/aggregate-functions/reference/{arrayconcatagg.md => array_concat_agg.md} (97%) diff --git a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md b/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md similarity index 97% rename from docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md rename to docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md index 50e4ed63787..db1f1a10859 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md @@ -6,7 +6,7 @@ sidebar_position: 110 # array_concat_agg - Alias of `groupArrayArray`. The function is case insensitive. -**Exampla** +**Example** ```text SELECT * diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 87d96f0d1ce..92484ad93ff 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -194,7 +194,7 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorMap(factory); } - { + { auto & factory = AggregateFunctionFactory::instance(); /// Must register after registerCombinator From 6899070f95ea81b6a77090f06fb45332f475d349 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:54:13 +0000 Subject: [PATCH 235/473] fix --- .../reference/{array_concat_agg.md => arrayconcatagg.md} | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) rename docs/en/sql-reference/aggregate-functions/reference/{array_concat_agg.md => arrayconcatagg.md} (86%) diff --git a/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md similarity index 86% rename from docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md rename to docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md index db1f1a10859..3c71129bdb5 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/aggregate-functions/reference/arrayconcatagg +slug: /en/sql-reference/aggregate-functions/reference/array_concat_agg sidebar_position: 110 --- diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..fc2cd5640b1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -990,6 +990,7 @@ addressToLine addressToLineWithInlines addressToSymbol adviced +agg aggregatefunction aggregatingmergetree aggregatio From 71d6206fc45e8622230ed2396f2662d76851eb37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 17:42:22 +0300 Subject: [PATCH 236/473] Fix Python --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 790adf9df62..abd109d00b2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1229,7 +1229,7 @@ class TestCase: .replace("\n", ", ") ) - if 0 != len(leftover_tables): + if len(leftover_tables) != 0: raise Exception( f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally." ) From bbade814f3e59bf31fb86cadb17b0ffed1661257 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 17:10:28 +0000 Subject: [PATCH 237/473] fix --- .../AggregateFunctionFactory.cpp | 28 ------------- .../AggregateFunctionFactory.h | 3 -- .../AggregateFunctionGroupArray.cpp | 1 + .../registerAggregateFunctions.cpp | 8 ---- src/Common/IFactoryWithAliases.h | 40 +++++++++---------- 5 files changed, 21 insertions(+), 59 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index f2dbb931b1f..f52c9ac8510 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -59,34 +59,6 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat } } -void AggregateFunctionFactory::registerAliasForAggregateFunctionWithCombinator( - const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) -{ - if (!isAggregateFunctionName(real_name)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "{}: can't create alias '{}', the real name '{}' is not registered", - getFactoryName(), - alias_name, - real_name); - - auto alias_name_lowercase = Poco::toLower(alias_name); - - if (aggregate_functions.contains(alias_name) || case_insensitive_aggregate_functions.contains(alias_name_lowercase)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", getFactoryName(), alias_name); - - if (case_sensitiveness == CaseInsensitive) - { - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", getFactoryName(), alias_name); - case_insensitive_name_mapping[alias_name_lowercase] = real_name; - } - - if (!aliases.emplace(alias_name, real_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", getFactoryName(), alias_name); -} - static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index 6c2b539bd8c..dab0d28e851 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -62,9 +62,6 @@ public: Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - void registerAliasForAggregateFunctionWithCombinator( - const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); - /// Throws an exception if not found. AggregateFunctionPtr get(const String & name, diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index bb1368b9ff8..a09e9ddfbf6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -125,6 +125,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); + factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 92484ad93ff..91248a52ae9 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -193,14 +193,6 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorDistinct(factory); registerAggregateFunctionCombinatorMap(factory); } - - { - auto & factory = AggregateFunctionFactory::instance(); - - /// Must register after registerCombinator - factory.registerAliasForAggregateFunctionWithCombinator( - "array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); - } } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index af5656ffb75..07440dd2463 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -10,8 +10,6 @@ namespace DB { -class AggregateFunctionFactory; - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -24,7 +22,6 @@ namespace ErrorCodes template class IFactoryWithAliases : public IHints<2, IFactoryWithAliases> { - friend AggregateFunctionFactory; protected: using Value = ValueType; @@ -55,35 +52,38 @@ public: { const auto & creator_map = getMap(); const auto & case_insensitive_creator_map = getCaseInsensitiveMap(); - const String factory_name = getFactoryName(); - String real_dict_name; - if (creator_map.contains(real_name)) - real_dict_name = real_name; - else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.contains(real_name_lowercase)) - real_dict_name = real_name_lowercase; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: can't create alias '{}', the real name '{}' is not registered", - factory_name, alias_name, real_name); + auto real_name_lowercase = Poco::toLower(real_name); + if (!creator_map.contains(real_name) && !case_insensitive_creator_map.contains(real_name_lowercase)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}: can't create alias '{}', the real name '{}' is not registered", + getFactoryName(), + alias_name, + real_name); + registerAliasUnchecked(alias_name, real_name, case_sensitiveness); + } + + /// We need sure the real_name exactly exists when call the function directly. + void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive) + { String alias_name_lowercase = Poco::toLower(alias_name); - - if (creator_map.contains(alias_name) || case_insensitive_creator_map.contains(alias_name_lowercase)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", - factory_name, alias_name); + String real_name_lowercase = Poco::toLower(real_name); + const String factory_name = getFactoryName(); if (case_sensitiveness == CaseInsensitive) { - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", - factory_name, alias_name); + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name); case_insensitive_name_mapping[alias_name_lowercase] = real_name; } - if (!aliases.emplace(alias_name, real_dict_name).second) + if (!aliases.emplace(alias_name, real_name).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", factory_name, alias_name); } + std::vector getAllRegisteredNames() const override { std::vector result; From a644317f577f6c5b573ddc99589ac8745d8e2e13 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 16 Jul 2023 18:07:22 +0000 Subject: [PATCH 238/473] Revert "Merge pull request #52129 from ClickHouse/revert-51291-ADQM-868" This reverts commit 1b9bcae68ee971a241b2a70b20408de095f1529f, reversing changes made to 7284749d05f31a687cd8091e1ffc249afd7c16b5. --- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 2 + src/Core/DecimalFunctions.h | 6 +- src/Functions/DateTimeTransforms.h | 71 ++++++++++-- src/Functions/TransformDateTime64.h | 7 +- src/Functions/dateDiff.cpp | 58 +++++++--- src/Functions/toStartOfInterval.cpp | 1 - ...0479_date_and_datetime_to_number.reference | 1 + .../00479_date_and_datetime_to_number.sql | 1 + .../02160_special_functions.reference | 10 ++ .../0_stateless/02160_special_functions.sql | 12 ++ .../02477_age_datetime64.reference | 106 ++++++++++++++++++ .../0_stateless/02477_age_datetime64.sql | 65 +++++++++++ 14 files changed, 323 insertions(+), 29 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 19eeda967fe..ce1a4f4d283 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -722,7 +722,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second. +Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond. E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit. For an alternative to `age`, see function `date\_diff`. @@ -738,6 +738,8 @@ age('unit', startdate, enddate, [timezone]) - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: + - `microsecond` (possible abbreviations: `us`, `u`) + - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) @@ -813,6 +815,8 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_ - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: + - `microsecond` (possible abbreviations: `us`, `u`) + - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 779728ca0fe..4db8a1ec6f8 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -625,7 +625,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду. +Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду. Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`. **Синтаксис** @@ -639,6 +639,8 @@ age('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: + - `microsecond` (возможные сокращения: `us`, `u`) + - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) @@ -712,6 +714,8 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: + - `microsecond` (возможные сокращения: `us`, `u`) + - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index 53dadc23c6d..e4b70322477 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -643,6 +643,8 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。 可能的值: + - `microsecond` + - `millisecond` - `second` - `minute` - `hour` diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 357cff2c541..17d95650730 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -48,7 +48,11 @@ inline auto scaleMultiplier(UInt32 scale) /** Components of DecimalX value: * whole - represents whole part of decimal, can be negative or positive. - * fractional - for fractional part of decimal, always positive. + * fractional - for fractional part of decimal. + * + * 0.123 represents 0 / 0.123 + * -0.123 represents 0 / -0.123 + * -1.123 represents -1 / 0.123 */ template struct DecimalComponents diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 510a88db2b6..a1c880f6956 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -19,6 +19,9 @@ namespace DB { +static constexpr auto microsecond_multiplier = 1000000; +static constexpr auto millisecond_multiplier = 1000; + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -1377,6 +1380,36 @@ struct ToRelativeSecondNumImpl using FactorTransform = ZeroTransform; }; +template +struct ToRelativeSubsecondNumImpl +{ + static constexpr auto name = "toRelativeSubsecondNumImpl"; + + static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) + { + static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000); + if (scale == scale_multiplier) + return t.value; + if (scale > scale_multiplier) + return t.value / (scale / scale_multiplier); + return t.value * (scale_multiplier / scale); + } + static inline Int64 execute(UInt32 t, const DateLUTImpl &) + { + return t * scale_multiplier; + } + static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) + { + return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; + } + static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone) + { + return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); + } + + using FactorTransform = ZeroTransform; +}; + struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; @@ -1476,25 +1509,47 @@ struct ToYYYYMMDDhhmmssImpl using FactorTransform = ZeroTransform; }; +struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeComponents +{ + UInt16 millisecond; + UInt16 microsecond; +}; + struct ToDateTimeComponentsImpl { static constexpr auto name = "toDateTimeComponents"; - static inline DateLUTImpl::DateTimeComponents execute(Int64 t, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(t); + auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + + if (t.value < 0 && components.fractional) + { + components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional; + --components.whole; + } + Int64 fractional = components.fractional; + if (scale_multiplier > microsecond_multiplier) + fractional = fractional / (scale_multiplier / microsecond_multiplier); + else if (scale_multiplier < microsecond_multiplier) + fractional = fractional * (microsecond_multiplier / scale_multiplier); + + constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier; + UInt16 millisecond = static_cast(fractional / divider); + UInt16 microsecond = static_cast(fractional % divider); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond}; } - static inline DateLUTImpl::DateTimeComponents execute(UInt32 t, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(static_cast(t)); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast(t)), 0, 0}; } - static inline DateLUTImpl::DateTimeComponents execute(Int32 d, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(ExtendedDayNum(d)); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0}; } - static inline DateLUTImpl::DateTimeComponents execute(UInt16 d, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(DayNum(d)); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0}; } using FactorTransform = ZeroTransform; diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index 3dab9efeb6b..fcee2753066 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -5,7 +5,7 @@ namespace DB { -/** Tansform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. +/** Transform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. * * Depending on what overloads of Transform::execute() are available, when called with DateTime64 value, * invokes Transform::execute() with either: @@ -80,7 +80,10 @@ public: } else { - const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + if (t.value < 0 && components.fractional) + --components.whole; + return wrapped_transform.execute(static_cast(components.whole), std::forward(args)...); } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 8361e9db166..6bfbbb7c735 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -174,12 +174,13 @@ public: { auto res = static_cast(transform_y.execute(y, timezone_y)) - static_cast(transform_x.execute(x, timezone_x)); - DateLUTImpl::DateTimeComponents a_comp; - DateLUTImpl::DateTimeComponents b_comp; + DateTimeComponentsWithFractionalPart a_comp; + DateTimeComponentsWithFractionalPart b_comp; Int64 adjust_value; - auto x_seconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); - auto y_seconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); - if (x_seconds <= y_seconds) + auto x_microseconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); + auto y_microseconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); + + if (x_microseconds <= y_microseconds) { a_comp = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, timezone_x); b_comp = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, timezone_y); @@ -192,14 +193,16 @@ public: adjust_value = 1; } + if constexpr (std::is_same_v>>) { if ((a_comp.date.month > b_comp.date.month) || ((a_comp.date.month == b_comp.date.month) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) - ))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -210,8 +213,9 @@ public: || ((x_month_in_quarter == y_month_in_quarter) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) - ))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -219,8 +223,9 @@ public: if ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) - ))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -230,25 +235,44 @@ public: if ((x_day_of_week > y_day_of_week) || ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour)) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { - if (a_comp.time.second > b_comp.time.second) + if ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))) + res += adjust_value; + } + else if constexpr (std::is_same_v>>) + { + if ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))) + res += adjust_value; + } + else if constexpr (std::is_same_v>>) + { + if (a_comp.microsecond > b_comp.microsecond) res += adjust_value; } return res; @@ -373,6 +397,10 @@ public: impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); + else if (unit == "millisecond" || unit == "ms") + impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); + else if (unit == "microsecond" || unit == "us" || unit == "u") + impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} does not support '{}' unit", getName(), unit); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 649242d0d86..48bf88cb14c 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 1375ccb1542..168b733d702 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,3 +4,4 @@ 201707 20170721 20170721112233 +19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 71151690028..1e35e99a802 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,3 +4,4 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 3a1dcd88902..5e7e3383d8d 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,4 +33,14 @@ Hello 2021-01-01 1 1 +86400000 +172800000 +86461000 +86401299 +701 +701 +800 +60200201 +60 +10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 6d18e7d0d25..64919536be3 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,4 +41,16 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); +SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); + +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); + SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index 3b4459dd26d..c8c716e1e9a 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,3 +111,109 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +5100200 +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); +5100200 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +5100 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); +5099 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +4 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +5 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); +4 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +3 +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +3 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); +1 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +16 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); +15 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +5 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); +4 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +8 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); +7 +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +2349 +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +2 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +5 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 1bed93991ca..889137395a3 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,3 +75,68 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); + +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); + +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); + +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); + +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); + +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); + +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); + +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); + +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); + +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file From abac46817ab9683f18ccabedeeb627d4f9652e20 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 16 Jul 2023 18:15:03 +0000 Subject: [PATCH 239/473] Move all tests to to one file --- ...0479_date_and_datetime_to_number.reference | 1 - .../00479_date_and_datetime_to_number.sql | 1 - .../02160_special_functions.reference | 10 -- .../0_stateless/02160_special_functions.sql | 12 -- .../02477_age_datetime64.reference | 106 --------------- .../0_stateless/02477_age_datetime64.sql | 65 --------- .../0_stateless/02814_age_datediff.reference | 128 ++++++++++++++++++ .../0_stateless/02814_age_datediff.sql | 79 +++++++++++ 8 files changed, 207 insertions(+), 195 deletions(-) create mode 100644 tests/queries/0_stateless/02814_age_datediff.reference create mode 100644 tests/queries/0_stateless/02814_age_datediff.sql diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 168b733d702..1375ccb1542 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,4 +4,3 @@ 201707 20170721 20170721112233 -19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 1e35e99a802..71151690028 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,4 +4,3 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 5e7e3383d8d..3a1dcd88902 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,14 +33,4 @@ Hello 2021-01-01 1 1 -86400000 -172800000 -86461000 -86401299 -701 -701 -800 -60200201 -60 -10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 64919536be3..6d18e7d0d25 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,16 +41,4 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); -SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); - -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); - SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index c8c716e1e9a..3b4459dd26d 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,109 +111,3 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -5100200 -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); -5100200 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -5100 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); -5099 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -4 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -5 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); -4 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -3 -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -3 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); -1 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -16 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); -15 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -5 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); -4 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -8 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); -7 --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -2349 -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -2 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -5 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 889137395a3..1bed93991ca 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,68 +75,3 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); - --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); - -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); - -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); - -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); - -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); - -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); - -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); - -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); - --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file diff --git a/tests/queries/0_stateless/02814_age_datediff.reference b/tests/queries/0_stateless/02814_age_datediff.reference new file mode 100644 index 00000000000..2b065211213 --- /dev/null +++ b/tests/queries/0_stateless/02814_age_datediff.reference @@ -0,0 +1,128 @@ +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +5100200 +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); +5100200 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +5100 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); +5099 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +4 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +5 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); +4 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +3 +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +3 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); +1 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +16 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); +15 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +5 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); +4 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +8 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); +7 +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +2349 +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +2 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +5 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +4 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); +86400000 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); +172800000 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); +86461000 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); +86401299 +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); +701 +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); +701 +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); +800 +SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); +60200201 +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +60 +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); +10 +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); +19691231235959 \ No newline at end of file diff --git a/tests/queries/0_stateless/02814_age_datediff.sql b/tests/queries/0_stateless/02814_age_datediff.sql new file mode 100644 index 00000000000..bc77f6de5d1 --- /dev/null +++ b/tests/queries/0_stateless/02814_age_datediff.sql @@ -0,0 +1,79 @@ + +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); + +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); + +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); + +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); + +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); + +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); + +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); + +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); + +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); + +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); +SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); + +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); + +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); \ No newline at end of file From 1c34d750143b3e4a659231116094a50e0e461327 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 16 Jul 2023 18:21:10 +0000 Subject: [PATCH 240/473] Add new lines --- tests/queries/0_stateless/02814_age_datediff.reference | 2 +- tests/queries/0_stateless/02814_age_datediff.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02814_age_datediff.reference b/tests/queries/0_stateless/02814_age_datediff.reference index 2b065211213..49165643657 100644 --- a/tests/queries/0_stateless/02814_age_datediff.reference +++ b/tests/queries/0_stateless/02814_age_datediff.reference @@ -125,4 +125,4 @@ SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970- SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); 10 SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); -19691231235959 \ No newline at end of file +19691231235959 diff --git a/tests/queries/0_stateless/02814_age_datediff.sql b/tests/queries/0_stateless/02814_age_datediff.sql index bc77f6de5d1..4d32af5d29d 100644 --- a/tests/queries/0_stateless/02814_age_datediff.sql +++ b/tests/queries/0_stateless/02814_age_datediff.sql @@ -76,4 +76,4 @@ SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021- SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); \ No newline at end of file +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); From 59ecdbd53ec9f8abaf841d4cb0410e93d03bb32e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 22:30:04 +0200 Subject: [PATCH 241/473] Fix test 02497_storage_file_reader_selection --- .../0_stateless/02497_storage_file_reader_selection.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index 8ea3adcbd2f..aa43e81f131 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -7,11 +7,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=test_02497_$CLICKHOUSE_TEST_UNIQUE_NAME.tsv echo -e 'key\nfoo\nbar' > $DATA_FILE -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap:" -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap:" && echo 1 || echo 'Fail' +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 'Fail' || echo 0 -$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" -$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" +$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap" && echo 'Fail' || echo 0 +$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 1 || echo 'Fail' $CLICKHOUSE_CLIENT --storage_file_read_method=mmap -nq "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }" From 422881ac2c27564fa0e7036c9a595b1f35237d45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:21:30 +0200 Subject: [PATCH 242/473] Fix logical error: cannot get name of not a column: Set --- src/Interpreters/evaluateConstantExpression.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 5a333172b14..f43b8bd168f 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -61,6 +62,11 @@ std::pair> evaluateConstantExpression(co ast->setAlias("constant_expression"); } + if (ast->as() != nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " + "is not a constant expression (unexpected AST node type): {}", ast->getID()); + ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters()); param_visitor.visit(ast); @@ -94,18 +100,18 @@ std::pair> evaluateConstantExpression(co if (!result_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Element of set in IN, VALUES or LIMIT or aggregate function parameter " + "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " "is not a constant expression (result column not found): {}", result_name); if (result_column->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty result column after evaluation " - "of constant expression for IN, VALUES or LIMIT or aggregate function parameter"); + "of constant expression for IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument"); /// Expressions like rand() or now() are not constant if (!isColumnConst(*result_column)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Element of set in IN, VALUES or LIMIT or aggregate function parameter " + "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " "is not a constant expression (result column is not const): {}", result_name); return std::make_pair((*result_column)[0], result_type); From de879a6004a9304596a274d6175af8a56a8b14fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:22:48 +0200 Subject: [PATCH 243/473] Add a test --- .../02815_logical_error_cannot_get_column_name_of_set.reference | 0 .../02815_logical_error_cannot_get_column_name_of_set.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.reference create mode 100644 tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.reference b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql new file mode 100644 index 00000000000..9d59f89a30a --- /dev/null +++ b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql @@ -0,0 +1 @@ +SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS } From 18bcbe400bdbc642995062ed5691d6ffe778127b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:26:21 +0200 Subject: [PATCH 244/473] Remove unused headers --- src/Interpreters/evaluateConstantExpression.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index f43b8bd168f..a2e17f5b834 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -15,14 +15,13 @@ #include #include #include -#include #include #include #include #include -#include #include + namespace DB { From ac408d0b63b8ec13c8b80ca65f887b21cf29795f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:10:42 +0200 Subject: [PATCH 245/473] Add a check for hasToken --- src/Functions/HasTokenImpl.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index fdec5fcb0b7..ab6b6399486 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -39,6 +39,9 @@ struct HasTokenImpl if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); + if (pattern.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); + if (haystack_offsets.empty()) return; From ace98464aef63545a23265fb71adc5147cfc0207 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:12:37 +0200 Subject: [PATCH 246/473] Add a test --- tests/queries/0_stateless/02816_has_token_empty.reference | 2 ++ tests/queries/0_stateless/02816_has_token_empty.sql | 7 +++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02816_has_token_empty.reference create mode 100644 tests/queries/0_stateless/02816_has_token_empty.sql diff --git a/tests/queries/0_stateless/02816_has_token_empty.reference b/tests/queries/0_stateless/02816_has_token_empty.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02816_has_token_empty.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02816_has_token_empty.sql b/tests/queries/0_stateless/02816_has_token_empty.sql new file mode 100644 index 00000000000..e5d6156debd --- /dev/null +++ b/tests/queries/0_stateless/02816_has_token_empty.sql @@ -0,0 +1,7 @@ +SELECT hasTokenCaseInsensitive('K(G', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenCaseInsensitive('Hello', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenCaseInsensitive('', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenCaseInsensitive('', 'Hello'); +SELECT hasToken('Hello', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasToken('', 'Hello'); +SELECT hasToken('', ''); -- { serverError BAD_ARGUMENTS } From e6f71e3ad2bd0b6c33772606783a9e225402f986 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:30:53 +0200 Subject: [PATCH 247/473] Fix error in groupArrayMovingSum --- .../AggregateFunctionGroupArrayMoving.cpp | 196 ++++++++++++++++- .../AggregateFunctionGroupArrayMoving.h | 207 ------------------ ...up_array_moving_zero_window_size.reference | 0 ...17_group_array_moving_zero_window_size.sql | 2 + 4 files changed, 195 insertions(+), 210 deletions(-) delete mode 100644 src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h create mode 100644 tests/queries/0_stateless/02817_group_array_moving_zero_window_size.reference create mode 100644 tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index e2acccce516..026b8d1956f 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -1,10 +1,25 @@ +#include #include -#include #include #include #include -#include #include +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include + +#include + +#define AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE 0xFFFFFF namespace DB @@ -13,11 +28,186 @@ struct Settings; namespace ErrorCodes { + extern const int TOO_LARGE_ARRAY_SIZE; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } +template +struct MovingData +{ + /// For easy serialization. + static_assert(std::has_unique_object_representations_v || std::is_floating_point_v); + + using Accumulator = T; + + /// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena + using Allocator = MixedAlignedArenaAllocator; + using Array = PODArray; + + Array value; /// Prefix sums. + T sum{}; + + void NO_SANITIZE_UNDEFINED add(T val, Arena * arena) + { + sum += val; + value.push_back(sum, arena); + } +}; + +template +struct MovingSumData : public MovingData +{ + static constexpr auto name = "groupArrayMovingSum"; + + T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const + { + if (idx < window_size) + return this->value[idx]; + else + return this->value[idx] - this->value[idx - window_size]; + } +}; + +template +struct MovingAvgData : public MovingData +{ + static constexpr auto name = "groupArrayMovingAvg"; + + T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const + { + if (idx < window_size) + return this->value[idx] / T(window_size); + else + return (this->value[idx] - this->value[idx - window_size]) / T(window_size); + } +}; + + +template +class MovingImpl final + : public IAggregateFunctionDataHelper> +{ + static constexpr bool limit_num_elems = LimitNumElements::value; + UInt64 window_size; + +public: + using ResultT = typename Data::Accumulator; + + using ColumnSource = ColumnVectorOrDecimal; + + /// Probably for overflow function in the future. + using ColumnResult = ColumnVectorOrDecimal; + + explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits::max()) + : IAggregateFunctionDataHelper>({data_type_}, {}, createResultType(data_type_)) + , window_size(window_size_) {} + + String getName() const override { return Data::name; } + + static DataTypePtr createResultType(const DataTypePtr & argument) + { + return std::make_shared(getReturnTypeElement(argument)); + } + + void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + auto value = static_cast(*columns[0]).getData()[row_num]; + this->data(place).add(static_cast(value), arena); + } + + void NO_SANITIZE_UNDEFINED merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & cur_elems = this->data(place); + auto & rhs_elems = this->data(rhs); + + size_t cur_size = cur_elems.value.size(); + + if (rhs_elems.value.size()) + cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); + + for (size_t i = cur_size; i < cur_elems.value.size(); ++i) + { + cur_elems.value[i] += cur_elems.sum; + } + + cur_elems.sum += rhs_elems.sum; + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + const auto & value = this->data(place).value; + size_t size = value.size(); + writeVarUInt(size, buf); + buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + { + size_t size = 0; + readVarUInt(size, buf); + + if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE); + + if (size > 0) + { + auto & value = this->data(place).value; + value.resize(size, arena); + buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); + this->data(place).sum = value.back(); + } + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + const auto & data = this->data(place); + size_t size = data.value.size(); + + ColumnArray & arr_to = assert_cast(to); + ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); + + offsets_to.push_back(offsets_to.back() + size); + + if (size) + { + typename ColumnResult::Container & data_to = assert_cast(arr_to.getData()).getData(); + + for (size_t i = 0; i < size; ++i) + { + if (!limit_num_elems) + { + data_to.push_back(data.get(i, size)); + } + else + { + data_to.push_back(data.get(i, window_size)); + } + } + } + } + + bool allocatesMemoryInArena() const override + { + return true; + } + +private: + static auto getReturnTypeElement(const DataTypePtr & argument) + { + if constexpr (!is_decimal) + return std::make_shared>(); + else + { + using Res = DataTypeDecimal; + return std::make_shared(Res::maxPrecision(), getDecimalScale(*argument)); + } + } +}; + + namespace { @@ -79,7 +269,7 @@ AggregateFunctionPtr createAggregateFunctionMoving( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || + if ((type == Field::Types::Int64 && parameters[0].get() <= 0) || (type == Field::Types::UInt64 && parameters[0].get() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h deleted file mode 100644 index e6f79d7bca1..00000000000 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ /dev/null @@ -1,207 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include - -#include -#include - -#include -#include - -#include - -#include - -#define AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE 0xFFFFFF - - -namespace DB -{ -struct Settings; - -namespace ErrorCodes -{ - extern const int TOO_LARGE_ARRAY_SIZE; -} - -template -struct MovingData -{ - /// For easy serialization. - static_assert(std::has_unique_object_representations_v || std::is_floating_point_v); - - using Accumulator = T; - - /// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena - using Allocator = MixedAlignedArenaAllocator; - using Array = PODArray; - - Array value; /// Prefix sums. - T sum{}; - - void NO_SANITIZE_UNDEFINED add(T val, Arena * arena) - { - sum += val; - value.push_back(sum, arena); - } -}; - -template -struct MovingSumData : public MovingData -{ - static constexpr auto name = "groupArrayMovingSum"; - - T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const - { - if (idx < window_size) - return this->value[idx]; - else - return this->value[idx] - this->value[idx - window_size]; - } -}; - -template -struct MovingAvgData : public MovingData -{ - static constexpr auto name = "groupArrayMovingAvg"; - - T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const - { - if (idx < window_size) - return this->value[idx] / T(window_size); - else - return (this->value[idx] - this->value[idx - window_size]) / T(window_size); - } -}; - - -template -class MovingImpl final - : public IAggregateFunctionDataHelper> -{ - static constexpr bool limit_num_elems = LimitNumElements::value; - UInt64 window_size; - -public: - using ResultT = typename Data::Accumulator; - - using ColumnSource = ColumnVectorOrDecimal; - - /// Probably for overflow function in the future. - using ColumnResult = ColumnVectorOrDecimal; - - explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper>({data_type_}, {}, createResultType(data_type_)) - , window_size(window_size_) {} - - String getName() const override { return Data::name; } - - static DataTypePtr createResultType(const DataTypePtr & argument) - { - return std::make_shared(getReturnTypeElement(argument)); - } - - void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override - { - auto value = static_cast(*columns[0]).getData()[row_num]; - this->data(place).add(static_cast(value), arena); - } - - void NO_SANITIZE_UNDEFINED merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override - { - auto & cur_elems = this->data(place); - auto & rhs_elems = this->data(rhs); - - size_t cur_size = cur_elems.value.size(); - - if (rhs_elems.value.size()) - cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); - - for (size_t i = cur_size; i < cur_elems.value.size(); ++i) - { - cur_elems.value[i] += cur_elems.sum; - } - - cur_elems.sum += rhs_elems.sum; - } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override - { - const auto & value = this->data(place).value; - size_t size = value.size(); - writeVarUInt(size, buf); - buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); - } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override - { - size_t size = 0; - readVarUInt(size, buf); - - if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, - "Too large array size (maximum: {})", AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE); - - if (size > 0) - { - auto & value = this->data(place).value; - value.resize(size, arena); - buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); - this->data(place).sum = value.back(); - } - } - - void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - { - const auto & data = this->data(place); - size_t size = data.value.size(); - - ColumnArray & arr_to = assert_cast(to); - ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); - - offsets_to.push_back(offsets_to.back() + size); - - if (size) - { - typename ColumnResult::Container & data_to = assert_cast(arr_to.getData()).getData(); - - for (size_t i = 0; i < size; ++i) - { - if (!limit_num_elems) - { - data_to.push_back(data.get(i, size)); - } - else - { - data_to.push_back(data.get(i, window_size)); - } - } - } - } - - bool allocatesMemoryInArena() const override - { - return true; - } - -private: - static auto getReturnTypeElement(const DataTypePtr & argument) - { - if constexpr (!is_decimal) - return std::make_shared>(); - else - { - using Res = DataTypeDecimal; - return std::make_shared(Res::maxPrecision(), getDecimalScale(*argument)); - } - } -}; - -#undef AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE - -} diff --git a/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.reference b/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql b/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql new file mode 100644 index 00000000000..fcbcaf1245b --- /dev/null +++ b/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql @@ -0,0 +1,2 @@ +SELECT groupArrayMovingAvg ( toInt64 ( 0 ) ) ( toDecimal32 ( 1 , 1 ) ); -- { serverError BAD_ARGUMENTS } + From 34ec39234f875d8309055b196fd083fc0fe74586 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 01:40:02 +0200 Subject: [PATCH 248/473] Fix crash in comparison functions due to incorrect query analysis --- src/Functions/FunctionsComparison.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 66269f72866..cd40880ba54 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1112,6 +1112,11 @@ private: bool c0_const = isColumnConst(*c0); bool c1_const = isColumnConst(*c1); + /// This is a paranoid check to protect from a broken query analysis. + if (c0->isNullable() != c1->isNullable()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Logical error: columns are assumed to be of identical types, but they are different in Nullable"); + if (c0_const && c1_const) { UInt8 res = 0; From 299149b68207294b5347ed1c2bcf82b68bfda9c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 01:49:43 +0200 Subject: [PATCH 249/473] Fix slow test 02317_distinct_in_order_optimization --- .../02317_distinct_in_order_optimization.sql | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index a794709caba..d05a25882a7 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -18,7 +18,7 @@ select distinct a from distinct_in_order settings max_block_size=10, max_threads select '-- create table with not only primary key columns'; drop table if exists distinct_in_order sync; -create table distinct_in_order (a int, b int, c int) engine=MergeTree() order by (a, b); +create table distinct_in_order (a int, b int, c int) engine=MergeTree() order by (a, b) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select number % number, number % 5, number % 10 from numbers(1,1000000); select '-- distinct with primary key prefix only'; @@ -59,16 +59,16 @@ drop table if exists distinct_in_order sync; select '-- check that distinct in order returns the same result as ordinary distinct'; drop table if exists distinct_cardinality_low sync; -CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); +CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; INSERT INTO distinct_cardinality_low SELECT number % 1e1, number % 1e2, number % 1e3 FROM numbers_mt(1e4); drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; select '-- check that distinct in order WITH order by returns the same result as ordinary distinct'; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; select count() as diff from (select distinct * from distinct_in_order except select * from ordinary_distinct); @@ -76,9 +76,9 @@ drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; select '-- check that distinct in order WITHOUT order by returns the same result as ordinary distinct'; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select distinct * from distinct_cardinality_low settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into ordinary_distinct select distinct * from distinct_cardinality_low settings optimize_distinct_in_order=0; select count() as diff from (select distinct * from distinct_in_order except select * from ordinary_distinct); @@ -86,9 +86,9 @@ drop table if exists distinct_in_order; drop table if exists ordinary_distinct; select '-- check that distinct in order WITHOUT order by and WITH filter returns the same result as ordinary distinct'; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select distinct * from distinct_cardinality_low where low > 0 settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into ordinary_distinct select distinct * from distinct_cardinality_low where low > 0 settings optimize_distinct_in_order=0; select count() as diff from (select distinct * from distinct_in_order except select * from ordinary_distinct); @@ -102,12 +102,12 @@ drop table if exists sorting_key_contain_function; select '-- bug 42185, distinct in order and empty sort description'; select '-- distinct in order, sorting key tuple()'; -create table sorting_key_empty_tuple (a int, b int) engine=MergeTree() order by tuple(); +create table sorting_key_empty_tuple (a int, b int) engine=MergeTree() order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into sorting_key_empty_tuple select number % 2, number % 5 from numbers(1,10); select distinct a from sorting_key_empty_tuple; select '-- distinct in order, sorting key contains function'; -create table sorting_key_contain_function (datetime DateTime, a int) engine=MergeTree() order by (toDate(datetime)); +create table sorting_key_contain_function (datetime DateTime, a int) engine=MergeTree() order by (toDate(datetime)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into sorting_key_contain_function values ('2000-01-01', 1); insert into sorting_key_contain_function values ('2000-01-01', 2); select distinct datetime from sorting_key_contain_function; From 4e727cdb2a870c6d0876ba0f251ccc161e0c9777 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 03:21:24 +0300 Subject: [PATCH 250/473] Update liftUpFunctions.cpp --- src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index 47b4e31ed32..34a1fc2bb88 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -31,7 +31,7 @@ namespace DB::QueryPlanOptimizations /// This is a check that output columns does not have the same name /// This is ok for DAG, but may introduce a bug in a SotringStep cause columns are selected by name. -static bool areOutputsAreConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) +static bool areOutputsConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) { std::unordered_set names; for (const auto & output : outputs) @@ -72,7 +72,7 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: if (unneeded_for_sorting->trivial()) return 0; - if (!areOutputsAreConvertableToBlock(needed_for_sorting->getOutputs())) + if (!areOutputsConvertableToBlock(needed_for_sorting->getOutputs())) return 0; // Sorting (parent_node) -> Expression (child_node) From e38a9de7c997f0f23456c978cdd991b9e579d6b3 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 17 Jul 2023 10:17:20 +0800 Subject: [PATCH 251/473] add aspell Signed-off-by: Lloyd-Pottiger --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..e6e8c273d25 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2539,3 +2539,4 @@ znode znodes zookeeperSessionUptime zstd +curdate From 58686d7850e32f7dc78a06297c4882685825f0ef Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 17 Jul 2023 10:23:39 +0800 Subject: [PATCH 252/473] add tests Signed-off-by: Lloyd-Pottiger --- .../queries/0_stateless/02813_func_now_and_alias.reference | 3 +++ tests/queries/0_stateless/02813_func_now_and_alias.sql | 3 +++ .../0_stateless/02813_func_today_and_alias.reference | 6 ++++++ tests/queries/0_stateless/02813_func_today_and_alias.sql | 6 ++++++ 4 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02813_func_now_and_alias.reference create mode 100644 tests/queries/0_stateless/02813_func_now_and_alias.sql create mode 100644 tests/queries/0_stateless/02813_func_today_and_alias.reference create mode 100644 tests/queries/0_stateless/02813_func_today_and_alias.sql diff --git a/tests/queries/0_stateless/02813_func_now_and_alias.reference b/tests/queries/0_stateless/02813_func_now_and_alias.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/02813_func_now_and_alias.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/02813_func_now_and_alias.sql b/tests/queries/0_stateless/02813_func_now_and_alias.sql new file mode 100644 index 00000000000..bcda1f26181 --- /dev/null +++ b/tests/queries/0_stateless/02813_func_now_and_alias.sql @@ -0,0 +1,3 @@ +SELECT now() = current_timestamp(); +SELECT now() = CURRENT_TIMESTAMP(); +SELECT now() = current_TIMESTAMP(); diff --git a/tests/queries/0_stateless/02813_func_today_and_alias.reference b/tests/queries/0_stateless/02813_func_today_and_alias.reference new file mode 100644 index 00000000000..a9e2f17562a --- /dev/null +++ b/tests/queries/0_stateless/02813_func_today_and_alias.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02813_func_today_and_alias.sql b/tests/queries/0_stateless/02813_func_today_and_alias.sql new file mode 100644 index 00000000000..d379aa74f18 --- /dev/null +++ b/tests/queries/0_stateless/02813_func_today_and_alias.sql @@ -0,0 +1,6 @@ +SELECT today() = current_date(); +SELECT today() = CURRENT_DATE(); +SELECT today() = current_DATE(); +SELECT today() = curdate(); +SELECT today() = CURDATE(); +SELECT today() = curDATE(); From 1cebf03cabd591adfd5cd9df600fbaae42551b21 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 10:41:18 +0800 Subject: [PATCH 253/473] add length alias --- src/Functions/array/length.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/array/length.cpp b/src/Functions/array/length.cpp index f09ad17892b..91a5e5fdec2 100644 --- a/src/Functions/array/length.cpp +++ b/src/Functions/array/length.cpp @@ -101,6 +101,7 @@ It is ok to have ASCII NUL bytes in strings, and they will be counted as well. .categories{"String", "Array"} }, FunctionFactory::CaseInsensitive); + factory.registerAlias("OCTET_LENGTH", "length", FunctionFactory::CaseInsensitive); } } From 7915ba253bbc786d1752db4f1f3206cf08631501 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 04:49:04 +0200 Subject: [PATCH 254/473] Better version --- src/Interpreters/ActionsVisitor.cpp | 8 ++------ src/Interpreters/evaluateConstantExpression.cpp | 8 -------- src/Parsers/ASTSetQuery.cpp | 10 ++++++++++ src/Parsers/ASTSetQuery.h | 3 +++ ...815_logical_error_cannot_get_column_name_of_set.sql | 2 ++ 5 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 01f2d4cf22e..efab11003f5 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -16,19 +16,15 @@ #include #include -#include #include #include #include #include -#include #include #include #include -#include #include -#include #include #include @@ -47,7 +43,6 @@ #include #include #include -#include #include #include #include @@ -61,6 +56,7 @@ #include #include + namespace DB { @@ -715,7 +711,7 @@ bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child) node->as()) return false; - /// Do not go to FROM, JOIN, UNION. + /// Do not go to FROM, JOIN, UNION if (child->as() || child->as()) return false; diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index a2e17f5b834..921cd5ff553 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -1,20 +1,17 @@ #include #include -#include #include #include #include #include #include -#include #include #include #include #include #include #include -#include #include #include #include @@ -61,11 +58,6 @@ std::pair> evaluateConstantExpression(co ast->setAlias("constant_expression"); } - if (ast->as() != nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " - "is not a constant expression (unexpected AST node type): {}", ast->getID()); - ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters()); param_visitor.visit(ast); diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 0b8d76dbb89..76ad812e713 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -64,4 +64,14 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma } } +void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const +{ + Hash hash = getTreeHash(); + + writeCString("__settings_", ostr); + writeText(hash.first, ostr); + ostr.write('_'); + writeText(hash.second, ostr); +} + } diff --git a/src/Parsers/ASTSetQuery.h b/src/Parsers/ASTSetQuery.h index 40abe2de31d..beed052c79a 100644 --- a/src/Parsers/ASTSetQuery.h +++ b/src/Parsers/ASTSetQuery.h @@ -37,6 +37,9 @@ public: void updateTreeHashImpl(SipHash & hash_state) const override; QueryKind getQueryKind() const override { return QueryKind::Set; } + + void appendColumnName(WriteBuffer & ostr) const override; + void appendColumnNameWithoutAlias(WriteBuffer & ostr) const override { return appendColumnName(ostr); } }; } diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql index 9d59f89a30a..98a8a8992b8 100644 --- a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql +++ b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql @@ -1 +1,3 @@ SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS } +SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION } +SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION } From 0cdb1d1466367c355660ff4b217e41d605b39dcf Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 10:52:05 +0800 Subject: [PATCH 255/473] add docs --- docs/en/sql-reference/functions/array-functions.md | 2 ++ docs/en/sql-reference/functions/string-functions.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 7f2b8f3c605..7aeb1af8f6b 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -102,6 +102,8 @@ The function also works for strings. Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT length(arr) FROM table` transforms to `SELECT arr.size0 FROM TABLE`. +Alias: `OCTET_LENGTH` + ## emptyArrayUInt8, emptyArrayUInt16, emptyArrayUInt32, emptyArrayUInt64 ## emptyArrayInt8, emptyArrayInt16, emptyArrayInt32, emptyArrayInt64 diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5175bbf0615..28dcf1c6b77 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -90,6 +90,8 @@ Returns the length of a string in bytes (not: in characters or Unicode code poin The function also works for arrays. +Alias: `OCTET_LENGTH` + ## lengthUTF8 Returns the length of a string in Unicode code points (not: in bytes or characters). It assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. From ebfc9c25f7e22bce3dc78ee5195799ea2c0ca3d8 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 10:59:40 +0800 Subject: [PATCH 256/473] add test --- tests/queries/0_stateless/02815_alias_to_length.reference | 2 ++ tests/queries/0_stateless/02815_alias_to_length.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02815_alias_to_length.reference create mode 100644 tests/queries/0_stateless/02815_alias_to_length.sql diff --git a/tests/queries/0_stateless/02815_alias_to_length.reference b/tests/queries/0_stateless/02815_alias_to_length.reference new file mode 100644 index 00000000000..7290ba859f4 --- /dev/null +++ b/tests/queries/0_stateless/02815_alias_to_length.reference @@ -0,0 +1,2 @@ +4 +4 diff --git a/tests/queries/0_stateless/02815_alias_to_length.sql b/tests/queries/0_stateless/02815_alias_to_length.sql new file mode 100644 index 00000000000..531bb92f8f0 --- /dev/null +++ b/tests/queries/0_stateless/02815_alias_to_length.sql @@ -0,0 +1,2 @@ +SELECT OCTET_LENGTH([1,2,3,4]); +SELECT OCTET_LENGTH('1234'); From 3df812f4f03fa8149a009cad3f6735d8d702627e Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 11:18:30 +0800 Subject: [PATCH 257/473] fix --- tests/queries/0_stateless/02815_alias_to_length.reference | 2 ++ tests/queries/0_stateless/02815_alias_to_length.sql | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02815_alias_to_length.reference b/tests/queries/0_stateless/02815_alias_to_length.reference index 7290ba859f4..de958a364ef 100644 --- a/tests/queries/0_stateless/02815_alias_to_length.reference +++ b/tests/queries/0_stateless/02815_alias_to_length.reference @@ -1,2 +1,4 @@ 4 4 +15 +4 diff --git a/tests/queries/0_stateless/02815_alias_to_length.sql b/tests/queries/0_stateless/02815_alias_to_length.sql index 531bb92f8f0..780ac7dac6d 100644 --- a/tests/queries/0_stateless/02815_alias_to_length.sql +++ b/tests/queries/0_stateless/02815_alias_to_length.sql @@ -1,2 +1,6 @@ -SELECT OCTET_LENGTH([1,2,3,4]); SELECT OCTET_LENGTH('1234'); +SELECT OcTet_lenGtH('1234'); +SELECT OCTET_LENGTH('你好,世界'); + +-- This is a implementation-specific behavior of getting the length of an array. +SELECT OCTET_LENGTH([1,2,3,4]); From 0353eb1f4578c88bbea0b6051a9c0db86cd19cc4 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 05:05:33 +0000 Subject: [PATCH 258/473] add timezone to datediff() to fix other bug --- tests/queries/0_stateless/02814_age_datediff.reference | 6 ++++-- tests/queries/0_stateless/02814_age_datediff.sql | 5 +++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02814_age_datediff.reference b/tests/queries/0_stateless/02814_age_datediff.reference index 49165643657..cbcb8c8a7b6 100644 --- a/tests/queries/0_stateless/02814_age_datediff.reference +++ b/tests/queries/0_stateless/02814_age_datediff.reference @@ -1,3 +1,5 @@ +-- { echo } + -- DateTime64 vs DateTime64 with fractional part SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); 5100200 @@ -120,9 +122,9 @@ SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02 800 SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); 60200201 -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6, 'UTC'), '1970-01-01 00:00:00.000010'::DateTime64(6, 'UTC')); 60 -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6, 'UTC'), '1970-01-01 00:00:09.123'::DateTime64(6, 'UTC')); 10 SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); 19691231235959 diff --git a/tests/queries/0_stateless/02814_age_datediff.sql b/tests/queries/0_stateless/02814_age_datediff.sql index 4d32af5d29d..934a95c035f 100644 --- a/tests/queries/0_stateless/02814_age_datediff.sql +++ b/tests/queries/0_stateless/02814_age_datediff.sql @@ -1,3 +1,4 @@ +-- { echo } -- DateTime64 vs DateTime64 with fractional part SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); @@ -73,7 +74,7 @@ SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021- SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6, 'UTC'), '1970-01-01 00:00:00.000010'::DateTime64(6, 'UTC')); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6, 'UTC'), '1970-01-01 00:00:09.123'::DateTime64(6, 'UTC')); SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); From 708b980d693c13362b9222a0bfa74d1ab691a9f3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 05:36:24 +0000 Subject: [PATCH 259/473] Fix arg name to fix build --- src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index bf41ff12d2a..3f5ad8a5b6c 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -109,7 +109,7 @@ public: const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_, - std::shared_ptr> ex_queue = nullptr); + std::shared_ptr> queue_ = nullptr); void shutdown() override; From 0917cf05881f65faddb950daee60f67f53448129 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 17 Jul 2023 08:16:58 +0000 Subject: [PATCH 260/473] disable direct join for range dictionary --- src/Interpreters/JoinedTables.cpp | 5 +++ .../02815_range_dict_no_direct_join.reference | 6 ++++ .../02815_range_dict_no_direct_join.sql | 33 +++++++++++++++++++ 3 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/02815_range_dict_no_direct_join.reference create mode 100644 tests/queries/0_stateless/02815_range_dict_no_direct_join.sql diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index ee5c288afbb..29add31fd5d 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -337,6 +337,11 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name); return nullptr; } + if (dictionary->getSpecialKeyType() == DictionarySpecialKeyType::Range) + { + LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' is a range dictionary", dictionary_name); + return nullptr; + } auto dictionary_kv = std::dynamic_pointer_cast(dictionary); table_join->setStorageJoin(dictionary_kv); diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference new file mode 100644 index 00000000000..3576991f7a4 --- /dev/null +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference @@ -0,0 +1,6 @@ +1 0.1 +1 0.2 +2 0.3 +2 0.4 +3 0.5 +3 0.6 diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql new file mode 100644 index 00000000000..249ffdf8089 --- /dev/null +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql @@ -0,0 +1,33 @@ +CREATE TABLE discounts +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +ENGINE = Memory; + +INSERT INTO discounts VALUES (1, '2015-01-01', Null, 0.1); +INSERT INTO discounts VALUES (1, '2015-01-15', Null, 0.2); +INSERT INTO discounts VALUES (2, '2015-01-01', '2015-01-15', 0.3); +INSERT INTO discounts VALUES (2, '2015-01-04', '2015-01-10', 0.4); +INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-15', 0.5); +INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-10', 0.6); + +CREATE DICTIONARY discounts_dict +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +PRIMARY KEY advertiser_id +SOURCE(CLICKHOUSE(TABLE discounts)) +LIFETIME(MIN 600 MAX 900) +LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'max')) +RANGE(MIN discount_start_date MAX discount_end_date); + +CREATE TABLE ids (id UInt64) ENGINE = Memory; +INSERT INTO ids SELECT * FROM numbers(10); + +SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id SETTINGS join_algorithm = 'direct'; \ No newline at end of file From dfc1cc13c8bb58fbb224e5ed0651fcdb3c10c36d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jul 2023 08:54:55 +0000 Subject: [PATCH 261/473] Better oldest part selection --- tests/integration/test_multiple_disks/test.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 5561d63840b..bc7ac6683af 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -818,9 +818,10 @@ def test_start_stop_moves(start_cluster, name, engine): node1.query(f"SYSTEM STOP MOVES {name}") node1.query(f"SYSTEM STOP MERGES {name}") + first_part = None for i in range(5): data = [] # 5MB in total - for i in range(5): + for _ in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row # jbod size is 40MB, so lets insert 5MB batch 7 times node1.query_with_retry( @@ -829,7 +830,13 @@ def test_start_stop_moves(start_cluster, name, engine): ) ) - first_part = get_oldest_part(node1, name) + # we cannot rely simply on modification time of part because it can be changed + # by different background operations so we explicitly check after the first + # part is inserted + if i == 0: + first_part = get_oldest_part(node1, name) + + assert first_part is not None used_disks = get_used_disks_for_table(node1, name) From 1e984da9e6bbebcb4a345247f006ae28d4ec605d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 11:26:58 +0200 Subject: [PATCH 262/473] Fix --- .../02240_filesystem_cache_bypass_cache_threshold.reference | 1 - .../02240_filesystem_cache_bypass_cache_threshold.sql | 3 --- .../queries/0_stateless/02240_filesystem_query_cache.reference | 1 - tests/queries/0_stateless/02240_filesystem_query_cache.sql | 3 +-- .../queries/0_stateless/02286_drop_filesystem_cache.reference | 2 ++ .../02503_cache_on_write_with_small_segment_size.sh | 1 - 6 files changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference index 4828d9771b3..c29d736fa84 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference @@ -17,7 +17,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, bypass_cache_threashold = 100, cache_on_write_operations = 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql index ae2cd1b8cd1..8871f8655dd 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql @@ -18,9 +18,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', enable_bypass_cache_with_threashold = 1, bypass_cache_threashold = 100, - cache_on_write_operations = 1, - enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.reference b/tests/queries/0_stateless/02240_filesystem_query_cache.reference index 2e59aea73b9..26340c271e1 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.reference +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.reference @@ -17,7 +17,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', cache_on_write_operations= 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); SYSTEM DROP FILESYSTEM CACHE; diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 02cf54b0caa..d85b3f543e1 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -8,7 +8,6 @@ SET skip_download_if_exceeds_query_cache=1; SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; -SYSTEM DROP FILESYSTEM CACHE; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key @@ -21,9 +20,9 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', cache_on_write_operations= 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.reference b/tests/queries/0_stateless/02286_drop_filesystem_cache.reference index 279e77e795a..b4e5b6715de 100644 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.reference +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.reference @@ -6,6 +6,7 @@ Using storage policy: s3_cache 1 1 1 +0 Using storage policy: local_cache 0 2 @@ -14,3 +15,4 @@ Using storage policy: local_cache 1 1 1 +0 diff --git a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh index 2c526d10cc9..229c68bf8ec 100755 --- a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh +++ b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh @@ -25,7 +25,6 @@ SETTINGS min_bytes_for_wide_part = 0, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', cache_on_write_operations = 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); From 09d8ca0a40464d328f0708c915af09ffe8b1914e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 11:30:21 +0200 Subject: [PATCH 263/473] Update --- tests/config/config.d/storage_conf.xml | 10 +--------- tests/queries/0_stateless/02801_backup_native_copy.sh | 2 +- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index bd99ff06fb1..8533fef9fc9 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -1,18 +1,10 @@ - - s3 - s3_common_disk/ - http://localhost:11111/test/common/ - clickhouse - clickhouse - 20000 - s3 s3_disk/ - http://localhost:11111/test/00170_test/ + http://localhost:11111/test/common/ clickhouse clickhouse 20000 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh index 015dcb19b82..31a7cc3b410 100755 --- a/tests/queries/0_stateless/02801_backup_native_copy.sh +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -10,7 +10,7 @@ set -e $CLICKHOUSE_CLIENT -nm -q " drop table if exists data; - create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_common_disk'; + create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_disk'; insert into data select * from numbers(10); " From 41bdcdabfab16e6f93b4e0371563b149a1f0209b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Jul 2023 12:02:32 +0200 Subject: [PATCH 264/473] Create ZK ancestors optimistically --- src/Common/ZooKeeper/ZooKeeper.cpp | 32 +++++++++++++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5dd7948276d..6f8c2105cad 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -3,8 +3,10 @@ #include "KeeperException.h" #include "TestKeeper.h" -#include #include +#include +#include +#include #include #include @@ -350,15 +352,33 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & void ZooKeeper::createAncestors(const std::string & path) { - size_t pos = 1; + std::string data = ""; + std::string path_created; // Ignored + std::vector pending_nodes; + + size_t last_pos = path.rfind('/'); + std::string current_node = path.substr(0, last_pos); + while (true) { - pos = path.find('/', pos); - if (pos == std::string::npos) + Coordination::Error code = createImpl(current_node, data, CreateMode::Persistent, path_created); + if (code == Coordination::Error::ZNONODE) + { + /// The parent node doesn't exist. Save the current node and try with the parent + last_pos = current_node.rfind('/'); + if (last_pos == std::string::npos || last_pos == 0) + throw KeeperException(code, path); + pending_nodes.emplace_back(std::move(current_node)); + current_node = path.substr(0, last_pos); + } + else if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) break; - createIfNotExists(path.substr(0, pos), ""); - ++pos; + else + throw KeeperException(code, path); } + + for (const std::string & pending : pending_nodes | std::views::reverse) + createIfNotExists(pending, data); } void ZooKeeper::checkExistsAndGetCreateAncestorsOps(const std::string & path, Coordination::Requests & requests) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..ed21e89a163 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9173,8 +9173,7 @@ std::optional StorageReplicatedMergeTree::tryCreateZeroCopyExclusi String zc_zookeeper_path = *getZeroCopyPartPath(part_name, disk); /// Just recursively create ancestors for lock - zookeeper->createAncestors(zc_zookeeper_path); - zookeeper->createIfNotExists(zc_zookeeper_path, ""); + zookeeper->createAncestors(zc_zookeeper_path + "/"); /// Create actual lock ZeroCopyLock lock(zookeeper, zc_zookeeper_path, replica_name); From 9677cc5e2bee02991cfc0fb9bb5945b69fc7abd5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jul 2023 12:50:02 +0200 Subject: [PATCH 265/473] Apply suggestions from code review --- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperReconfiguration.cpp | 4 ++-- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/RaftServerConfig.cpp | 4 +++- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 40f1dac1570..331e3d2e73e 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -135,7 +135,7 @@ public: bool isServerActive() const; void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); - void pushClusterUpdates(ClusterUpdateActions&& actions); + void pushClusterUpdates(ClusterUpdateActions && actions); bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index f262a07209d..e3642913a7a 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -39,7 +39,7 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: if (leaving_arr.size() >= cfg->get_servers().size()) return {}; - std::unordered_set remove_ids; + std::unordered_set remove_ids; ClusterUpdateActions out; for (std::string_view leaving_server : leaving_arr) @@ -65,7 +65,7 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates) { RaftServers new_config; - std::unordered_set remove_update_ids; + std::unordered_set remove_update_ids; for (const auto & update : updates) { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index fd82f220f9b..a4c3d91e1c9 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -789,7 +789,7 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) +bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) { std::lock_guard _{server_write_mutex}; diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 9090ed68fb6..929eeeb640e 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -69,11 +69,12 @@ std::optional RaftServerConfig::parse(std::string_view server) RaftServers parseRaftServers(std::string_view servers) { std::vector server_arr; - std::unordered_set ids; + std::unordered_set ids; std::unordered_set endpoints; RaftServers out; for (auto & server : splitInto<','>(server_arr, servers)) + { if (auto maybe_server = RaftServerConfig::parse(server)) { String endpoint = maybe_server->endpoint; @@ -89,6 +90,7 @@ RaftServers parseRaftServers(std::string_view servers) } else return {}; + } return out; } From 477b8f04b61372e5f79d60f1efb08eb42d6508a4 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 13:09:34 +0200 Subject: [PATCH 266/473] Fix sticky mutations test (and extremely rare race condition) --- src/Storages/StorageMergeTree.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 074f01e7d03..8cc2fbc2911 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -599,7 +599,14 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer /// Validate partition IDs (if any) before starting mutation getPartitionIdsAffectedByCommands(commands, query_context); - Int64 version = startMutation(commands, query_context); + Int64 version; + { + /// It's important to serialize order of mutations with alter queries because + /// they can depend on each other. + auto alter_lock = lockForAlter(query_context->getSettings().lock_acquire_timeout); + version = startMutation(commands, query_context); + } + if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) waitForMutation(version, false); } From c374653c7bfc7450590d6a787bd4f413d15a76af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:01:47 +0200 Subject: [PATCH 267/473] Revert "Merge pull request #52138 from azat/decompressor-inode" This reverts commit 6524031348c3a1148e27ef926baabb35e833a09c, reversing changes made to 9bf114f9a36556d6f227dea0fa91be131ee99710. This was not a good idea, since the underlying problem was that `/proc/self/exe` was pointing to `qemu-$ARCH-static` (because the code uses realpath() over normal interface readlink(), which is not caught by the qemu linux-user). And this means that later, it will try to overwrite incorrect binary and then execute some garbage. Signed-off-by: Azat Khuzhin --- .../decompressor.cpp | 93 +++++++++---------- 1 file changed, 46 insertions(+), 47 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 4a4985120fd..d41b9b1ebe1 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -430,58 +430,55 @@ int main(int/* argc*/, char* argv[]) return 1; } - int lock = -1; - /// Protection from double decompression #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); - /// In some cases /proc/self/maps may not contain the inode for the - /// /proc/self/exe, one of such examples are using qemu-*-static, in this - /// case maps will be proxied through the qemu, and it will remove - /// information about itself from it. - if (inode != 0) + if (inode == 0) { - std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; - lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); - if (lock < 0) + std::cerr << "Unable to obtain inode." << std::endl; + return 1; + } + + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) + { + perror("lock open"); + return 1; + } + + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); + + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to + /// 32bit conversion of input_info.st_ino + if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) + input_info.st_ino &= 0x00000000FFFFFFFF; + + /// if decompression was performed by another process since this copy was started + /// then file referred by path "self" is already pointing to different inode + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) { - perror("lock open"); + perror("fstat lock"); return 1; } - /// lock file should be closed on exec call - fcntl(lock, F_SETFD, FD_CLOEXEC); + /// size 1 of lock file indicates that another decompressor has found active executable + if (lock_info.st_size == 1) + execv(self, argv); - if (lockf(lock, F_LOCK, 0)) - { - perror("lockf"); - return 1; - } - - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to - /// 32bit conversion of input_info.st_ino - if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) - input_info.st_ino &= 0x00000000FFFFFFFF; - - /// if decompression was performed by another process since this copy was started - /// then file referred by path "self" is already pointing to different inode - if (input_info.st_ino != inode) - { - struct stat lock_info; - if (0 != fstat(lock, &lock_info)) - { - perror("fstat lock"); - return 1; - } - - /// size 1 of lock file indicates that another decompressor has found active executable - if (lock_info.st_size == 1) - execv(self, argv); - - printf("No target executable - decompression only was performed.\n"); - return 0; - } + printf("No target executable - decompression only was performed.\n"); + return 0; } #endif @@ -549,19 +546,21 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed - if (lock >= 0) - write(lock, "1", 1); + write(lock, "1", 1); +#endif execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 - if (lock >= 0) - ftruncate(lock, 0); + ftruncate(lock, 0); +#endif printf("No target executable - decompression only was performed.\n"); } From 16165d9498cbebd3ecd02df87480230fd0ed880e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:02:54 +0200 Subject: [PATCH 268/473] Improve error messages for decompressor Signed-off-by: Azat Khuzhin --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index d41b9b1ebe1..567d9088f13 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -435,7 +435,7 @@ int main(int/* argc*/, char* argv[]) uint64_t inode = getInode(self); if (inode == 0) { - std::cerr << "Unable to obtain inode." << std::endl; + std::cerr << "Unable to obtain inode for exe '" << self << "'." << std::endl; return 1; } From 1fb7605fb4225225d492cf63f7f048d594e89fc3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:09:11 +0200 Subject: [PATCH 269/473] Fix self extracting binaries under qemu linux-user (qemu-$ARCH-static) The problem was that the decompressor uses realpath(/proc/self/exe) instead of readlink(/proc/self/exe), while realpath() does lots of trickerly [1] which leads to bypassing qemu linux-user override [2] of /proc/self/exe to the executable with with it had been called -- and the reason for this is that the getpid() after unshare returns 1, while reading /proc/self returns the pid that was before unshare (from the chroot) [3]. [1]: https://github.com/bminor/glibc/blob/4290aed05135ae4c0272006442d147f2155e70d7/stdlib/canonicalize.c#L223 [2]: https://github.com/qemu/qemu/blob/ed8ad9728a9c0eec34db9dff61dfa2f1dd625637/linux-user/syscall.c#L8634 [3]: https://gist.github.com/azat/fcbd8b6c26afd505ae5f3387fc15f0e2 But note, that even after this patch qemu without binfmt will not work, due to internally the code calls execv() while qemu does not handle it (see [4]). [4]: https://patchwork.kernel.org/project/qemu-devel/patch/1453091602-21843-1-git-send-email-petrosagg@gmail.com/ Signed-off-by: Azat Khuzhin --- utils/self-extracting-executable/decompressor.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 567d9088f13..91f4bea5a5b 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -362,11 +362,12 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #else - int read_exe_path(char *exe, size_t/* buf_sz*/) + int read_exe_path(char *exe, size_t buf_sz) { - if (realpath("/proc/self/exe", exe) == nullptr) - return 1; - return 0; + ssize_t n = readlink("/proc/self/exe", exe, buf_sz - 1); + if (n > 0) + exe[n] = '\0'; + return n > 0 && n < static_cast(buf_sz); } #endif From 5b853c19580def53df4add0ca6242d5912a7bfe3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 17 Jul 2023 11:32:22 +0000 Subject: [PATCH 270/473] Make ASTFuzzer happy --- src/Parsers/ASTCreateIndexQuery.cpp | 3 +-- src/Parsers/ASTIndexDeclaration.cpp | 9 +++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTCreateIndexQuery.cpp b/src/Parsers/ASTCreateIndexQuery.cpp index 50470fbc1e4..0d580d5bb21 100644 --- a/src/Parsers/ASTCreateIndexQuery.cpp +++ b/src/Parsers/ASTCreateIndexQuery.cpp @@ -56,8 +56,7 @@ void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, Forma formatOnCluster(settings); - if (!cluster.empty()) - settings.ostr << " "; + settings.ostr << " "; index_decl->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index e647f02f553..12d59681cc3 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -29,9 +29,14 @@ void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & sta { if (part_of_create_index_query) { - s.ostr << "("; + if (expr->as()) + { + s.ostr << "("; + expr->formatImpl(s, state, frame); + s.ostr << ")"; + } + else expr->formatImpl(s, state, frame); - s.ostr << ")"; } else { From 10c15a6548a27bbb51eb04082a11fd110a4642a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 10:33:09 +0000 Subject: [PATCH 271/473] Refactor limit_jobs.cmake --- cmake/limit_jobs.cmake | 53 ++++++++++++++++++++---------------------- 1 file changed, 25 insertions(+), 28 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 3a33b3b9989..b2a54fb8d1a 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -1,16 +1,14 @@ -# Usage: -# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # In megabytes -# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +# Usage from CMake: +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte # include (cmake/limit_jobs.cmake) cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) # Not available under freebsd cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) -# 1 if not set -option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" "") - -# 1 if not set -option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" "") +# Set to disable the automatic job-limiting +option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" OFF) +option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" OFF) if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) math(EXPR PARALLEL_COMPILE_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) @@ -19,17 +17,10 @@ if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_COMPILE_JOBS_LESS TRUE) + set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) endif() endif () -if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) - set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) - string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) -endif () - - if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) math(EXPR PARALLEL_LINK_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) @@ -37,7 +28,7 @@ if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_LINK_JOBS_LESS TRUE) + set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -52,20 +43,26 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() +message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") + +if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) + if (PARALLEL_COMPILE_JOBS_UNDERUTILIZING) + message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + endif() + if (PARALLEL_LINK_JOBS_UNDERUTILIZING) + message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + endif() +endif () + +if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) + set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) + string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) +endif () + if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () -if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - message(STATUS - "${CMAKE_CURRENT_SOURCE_DIR}: Have ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. - Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS} (system has ${NUMBER_OF_LOGICAL_CORES} logical cores)") - if (PARALLEL_COMPILE_JOBS_LESS) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") - endif() - if (PARALLEL_LINK_JOBS_LESS) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") - endif() -endif () From a110c879852792739182ea9bb1fe55ff193d457f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 17 Jul 2023 13:46:04 +0200 Subject: [PATCH 272/473] Update 02240_filesystem_cache_bypass_cache_threshold.reference --- .../02240_filesystem_cache_bypass_cache_threshold.reference | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference index c29d736fa84..eb6c9305ebd 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference @@ -15,8 +15,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', enable_bypass_cache_with_threashold = 1, bypass_cache_threashold = 100, - cache_on_write_operations = 1, - enable_filesystem_query_cache_limit = 1, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); From 7204f7054788302e37be3e79ddd3a0ff87b93f01 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:52:00 +0000 Subject: [PATCH 273/473] cmake_host_system_information provides TOTAL_PHYSICAL_MEMORY on all platforms, including FreeBSD --- cmake/limit_jobs.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b2a54fb8d1a..2ca9e5519d7 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -3,14 +3,14 @@ # set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte # include (cmake/limit_jobs.cmake) -cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) # Not available under freebsd +cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) # Set to disable the automatic job-limiting option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" OFF) option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" OFF) -if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) +if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) math(EXPR PARALLEL_COMPILE_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) if (NOT PARALLEL_COMPILE_JOBS) @@ -21,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) endif() endif () -if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) +if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) math(EXPR PARALLEL_LINK_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) if (NOT PARALLEL_LINK_JOBS) From 8f1e56167c430783841abca00dd16c217cfcec13 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:53:12 +0000 Subject: [PATCH 274/473] NUMBER_OF_TOTAL_CORES is on all platforms != 0 --- cmake/limit_jobs.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 2ca9e5519d7..b145c069e37 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -16,7 +16,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) if (NOT PARALLEL_COMPILE_JOBS) set (PARALLEL_COMPILE_JOBS 1) endif () - if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) + if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -27,7 +27,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) if (NOT PARALLEL_LINK_JOBS) set (PARALLEL_LINK_JOBS 1) endif () - if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) + if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -54,13 +54,13 @@ if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) endif() endif () -if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () -if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_LINK_JOBS AND (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) From 496aaf58c8380efcb8ad7f46f470831269b6a95f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Jul 2023 13:55:14 +0200 Subject: [PATCH 275/473] Make clang-tidy happy --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 6f8c2105cad..e3e89969c17 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -352,7 +352,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & void ZooKeeper::createAncestors(const std::string & path) { - std::string data = ""; + std::string data; std::string path_created; // Ignored std::vector pending_nodes; From 96c399ff7b623cbebcf7a8e1ae036df5b0f57767 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:58:18 +0000 Subject: [PATCH 276/473] Simplify a bit further --- cmake/limit_jobs.cmake | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b145c069e37..b7cc5cf0967 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -17,7 +17,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) + message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -28,7 +28,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) + message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () @@ -45,15 +45,6 @@ endif() message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") -if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - if (PARALLEL_COMPILE_JOBS_UNDERUTILIZING) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") - endif() - if (PARALLEL_LINK_JOBS_UNDERUTILIZING) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") - endif() -endif () - if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) From 152f5527a906d048df2e727e3e592735f8844508 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jul 2023 11:58:22 +0000 Subject: [PATCH 277/473] Remove wrong assert --- tests/integration/ci-runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 9629a5821b5..31f4a7666c8 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -810,7 +810,6 @@ class ClickhouseIntegrationTestsRunner: result_state = "failure" if not should_fail: break - assert should_fail logging.info("Try is OK, all tests passed, going to clear env") clear_ip_tables_and_restart_daemons() logging.info("And going to sleep for some time") From 2993cf3cba6d9d4653cf4981e7fb4235a3a8acfa Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 14:01:31 +0200 Subject: [PATCH 278/473] Fix flaky test test_lost_part --- tests/integration/test_lost_part/test.py | 410 ++++++++++++----------- 1 file changed, 207 insertions(+), 203 deletions(-) diff --git a/tests/integration/test_lost_part/test.py b/tests/integration/test_lost_part/test.py index 0bc24268040..b83707145f4 100644 --- a/tests/integration/test_lost_part/test.py +++ b/tests/integration/test_lost_part/test.py @@ -39,257 +39,261 @@ def test_lost_part_same_replica(start_cluster): node1.query("DROP TABLE IF EXISTS mt0 SYNC") node2.query("DROP TABLE IF EXISTS mt0 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{node.name}') ORDER BY tuple() PARTITION BY date " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" - ) + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{node.name}') ORDER BY tuple() PARTITION BY date " + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) - node1.query("SYSTEM STOP MERGES mt0") - node2.query("SYSTEM STOP REPLICATION QUEUES") + node1.query("SYSTEM STOP MERGES mt0") + node2.query("SYSTEM STOP REPLICATION QUEUES") - for i in range(5): - node1.query(f"INSERT INTO mt0 VALUES ({i}, toDate('2020-10-01'))") + for i in range(5): + node1.query(f"INSERT INTO mt0 VALUES ({i}, toDate('2020-10-01'))") - for i in range(20): - parts_to_merge = node1.query( - "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt0' AND length(parts_to_merge) > 0" - ) - if parts_to_merge: - parts_list = list(sorted(ast.literal_eval(parts_to_merge))) - print("Got parts list", parts_list) - if len(parts_list) < 3: - raise Exception(f"Got too small parts list {parts_list}") - break - time.sleep(1) + for i in range(20): + parts_to_merge = node1.query( + "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt0' AND length(parts_to_merge) > 0" + ) + if parts_to_merge: + parts_list = list(sorted(ast.literal_eval(parts_to_merge))) + print("Got parts list", parts_list) + if len(parts_list) < 3: + raise Exception(f"Got too small parts list {parts_list}") + break + time.sleep(1) - victim_part_from_the_middle = random.choice(parts_list[1:-1]) - print("Will corrupt part", victim_part_from_the_middle) + victim_part_from_the_middle = random.choice(parts_list[1:-1]) + print("Will corrupt part", victim_part_from_the_middle) - remove_part_from_disk(node1, "mt0", victim_part_from_the_middle) + remove_part_from_disk(node1, "mt0", victim_part_from_the_middle) - node1.query("DETACH TABLE mt0") + node1.query("DETACH TABLE mt0") - node1.query("ATTACH TABLE mt0") + node1.query("ATTACH TABLE mt0") - node1.query("SYSTEM START MERGES mt0") - res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt0") - print("result: ", res) - print("error: ", res) + node1.query("SYSTEM START MERGES mt0") + res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt0") + print("result: ", res) + print("error: ", res) - for i in range(10): - result = node1.query("SELECT count() FROM system.replication_queue") - if int(result) == 0: - break - time.sleep(1) - else: - assert False, "Still have something in replication queue:\n" + node1.query( - "SELECT count() FROM system.replication_queue FORMAT Vertical" - ) + for i in range(10): + result = node1.query("SELECT count() FROM system.replication_queue") + if int(result) == 0: + break + time.sleep(1) + else: + assert False, "Still have something in replication queue:\n" + node1.query( + "SELECT count() FROM system.replication_queue FORMAT Vertical" + ) - assert node1.contains_in_log( - "Created empty part" - ), f"Seems like empty part {victim_part_from_the_middle} is not created or log message changed" + assert node1.contains_in_log( + "Created empty part" + ), f"Seems like empty part {victim_part_from_the_middle} is not created or log message changed" - assert node1.query("SELECT COUNT() FROM mt0") == "4\n" + assert node1.query("SELECT COUNT() FROM mt0") == "4\n" - node2.query("SYSTEM START REPLICATION QUEUES") + node2.query("SYSTEM START REPLICATION QUEUES") - assert_eq_with_retry(node2, "SELECT COUNT() FROM mt0", "4") - assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt0 SYNC") - node2.query("DROP TABLE IF EXISTS mt0 SYNC") + assert_eq_with_retry(node2, "SELECT COUNT() FROM mt0", "4") + assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt0 SYNC") + node2.query("DROP TABLE IF EXISTS mt0 SYNC") def test_lost_part_other_replica(start_cluster): node1.query("DROP TABLE IF EXISTS mt1 SYNC") node2.query("DROP TABLE IF EXISTS mt1 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{node.name}') ORDER BY tuple() " + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) + + node1.query("SYSTEM STOP MERGES mt1") + node2.query("SYSTEM STOP REPLICATION QUEUES") + + for i in range(5): + node1.query(f"INSERT INTO mt1 VALUES ({i})") + + for i in range(20): + parts_to_merge = node1.query( + "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt1' AND length(parts_to_merge) > 0" + ) + if parts_to_merge: + parts_list = list(sorted(ast.literal_eval(parts_to_merge))) + print("Got parts list", parts_list) + if len(parts_list) < 3: + raise Exception("Got too small parts list {}".format(parts_list)) + break + time.sleep(1) + + victim_part_from_the_middle = random.choice(parts_list[1:-1]) + print("Will corrupt part", victim_part_from_the_middle) + + remove_part_from_disk(node1, "mt1", victim_part_from_the_middle) + + # other way to detect broken parts + node1.query("CHECK TABLE mt1") + + node2.query("SYSTEM START REPLICATION QUEUES") + res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt1") + print("result: ", res) + print("error: ", res) + + for i in range(10): + result = node2.query("SELECT count() FROM system.replication_queue") + if int(result) == 0: + break + time.sleep(1) + else: + assert False, "Still have something in replication queue:\n" + node2.query( + "SELECT * FROM system.replication_queue FORMAT Vertical" + ) + + assert node1.contains_in_log( + "Created empty part" + ), "Seems like empty part {} is not created or log message changed".format( + victim_part_from_the_middle ) - node1.query("SYSTEM STOP MERGES mt1") - node2.query("SYSTEM STOP REPLICATION QUEUES") + assert_eq_with_retry(node2, "SELECT COUNT() FROM mt1", "4") + assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - for i in range(5): - node1.query(f"INSERT INTO mt1 VALUES ({i})") + node1.query("SYSTEM START MERGES mt1") - for i in range(20): - parts_to_merge = node1.query( - "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt1' AND length(parts_to_merge) > 0" - ) - if parts_to_merge: - parts_list = list(sorted(ast.literal_eval(parts_to_merge))) - print("Got parts list", parts_list) - if len(parts_list) < 3: - raise Exception("Got too small parts list {}".format(parts_list)) - break - time.sleep(1) - - victim_part_from_the_middle = random.choice(parts_list[1:-1]) - print("Will corrupt part", victim_part_from_the_middle) - - remove_part_from_disk(node1, "mt1", victim_part_from_the_middle) - - # other way to detect broken parts - node1.query("CHECK TABLE mt1") - - node2.query("SYSTEM START REPLICATION QUEUES") - res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt1") - print("result: ", res) - print("error: ", res) - - for i in range(10): - result = node2.query("SELECT count() FROM system.replication_queue") - if int(result) == 0: - break - time.sleep(1) - else: - assert False, "Still have something in replication queue:\n" + node2.query( - "SELECT * FROM system.replication_queue FORMAT Vertical" - ) - - assert node1.contains_in_log( - "Created empty part" - ), "Seems like empty part {} is not created or log message changed".format( - victim_part_from_the_middle - ) - - assert_eq_with_retry(node2, "SELECT COUNT() FROM mt1", "4") - assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("SYSTEM START MERGES mt1") - - assert_eq_with_retry(node1, "SELECT COUNT() FROM mt1", "4") - assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt1 SYNC") - node2.query("DROP TABLE IF EXISTS mt1 SYNC") + assert_eq_with_retry(node1, "SELECT COUNT() FROM mt1", "4") + assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt1 SYNC") + node2.query("DROP TABLE IF EXISTS mt1 SYNC") def test_lost_part_mutation(start_cluster): node1.query("DROP TABLE IF EXISTS mt2 SYNC") node2.query("DROP TABLE IF EXISTS mt2 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{node.name}') ORDER BY tuple() " + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) + + node1.query("SYSTEM STOP MERGES mt2") + node2.query("SYSTEM STOP REPLICATION QUEUES") + + for i in range(2): + node1.query(f"INSERT INTO mt2 VALUES ({i})") + + node1.query( + "ALTER TABLE mt2 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} ) - node1.query("SYSTEM STOP MERGES mt2") - node2.query("SYSTEM STOP REPLICATION QUEUES") + for i in range(20): + parts_to_mutate = node1.query( + "SELECT count() FROM system.replication_queue WHERE table='mt2'" + ) + # two mutations for both replicas + if int(parts_to_mutate) == 4: + break + time.sleep(1) - for i in range(2): - node1.query(f"INSERT INTO mt2 VALUES ({i})") + remove_part_from_disk(node1, "mt2", "all_1_1_0") - node1.query( - "ALTER TABLE mt2 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} - ) + # other way to detect broken parts + node1.query("CHECK TABLE mt2") - for i in range(20): - parts_to_mutate = node1.query( - "SELECT count() FROM system.replication_queue WHERE table='mt2'" - ) - # two mutations for both replicas - if int(parts_to_mutate) == 4: - break - time.sleep(1) + node1.query("SYSTEM START MERGES mt2") + res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt2") + print("result: ", res) + print("error: ", res) - remove_part_from_disk(node1, "mt2", "all_1_1_0") + for i in range(10): + result = node1.query("SELECT count() FROM system.replication_queue") + if int(result) == 0: + break + time.sleep(1) + else: + assert False, "Still have something in replication queue:\n" + node1.query( + "SELECT * FROM system.replication_queue FORMAT Vertical" + ) - # other way to detect broken parts - node1.query("CHECK TABLE mt2") + assert_eq_with_retry(node1, "SELECT COUNT() FROM mt2", "1") + assert_eq_with_retry(node1, "SELECT SUM(id) FROM mt2", "777") + assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - node1.query("SYSTEM START MERGES mt2") - res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt2") - print("result: ", res) - print("error: ", res) + node2.query("SYSTEM START REPLICATION QUEUES") - for i in range(10): - result = node1.query("SELECT count() FROM system.replication_queue") - if int(result) == 0: - break - time.sleep(1) - else: - assert False, "Still have something in replication queue:\n" + node1.query( - "SELECT * FROM system.replication_queue FORMAT Vertical" - ) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM mt2", "1") - assert_eq_with_retry(node1, "SELECT SUM(id) FROM mt2", "777") - assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - - node2.query("SYSTEM START REPLICATION QUEUES") - - assert_eq_with_retry(node2, "SELECT COUNT() FROM mt2", "1") - assert_eq_with_retry(node2, "SELECT SUM(id) FROM mt2", "777") - assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt2 SYNC") - node2.query("DROP TABLE IF EXISTS mt2 SYNC") + assert_eq_with_retry(node2, "SELECT COUNT() FROM mt2", "1") + assert_eq_with_retry(node2, "SELECT SUM(id) FROM mt2", "777") + assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt2 SYNC") + node2.query("DROP TABLE IF EXISTS mt2 SYNC") def test_lost_last_part(start_cluster): node1.query("DROP TABLE IF EXISTS mt3 SYNC") node2.query("DROP TABLE IF EXISTS mt3 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{node.name}') " - "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{node.name}') " + "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) + + node1.query("SYSTEM STOP MERGES mt3") + node2.query("SYSTEM STOP REPLICATION QUEUES") + + for i in range(1): + node1.query(f"INSERT INTO mt3 VALUES ({i}, 'x')") + + # actually not important + node1.query( + "ALTER TABLE mt3 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} ) - node1.query("SYSTEM STOP MERGES mt3") - node2.query("SYSTEM STOP REPLICATION QUEUES") + partition_id = node1.query("select partitionId('x')").strip() + remove_part_from_disk(node1, "mt3", f"{partition_id}_0_0_0") - for i in range(1): - node1.query(f"INSERT INTO mt3 VALUES ({i}, 'x')") + # other way to detect broken parts + node1.query("CHECK TABLE mt3") - # actually not important - node1.query( - "ALTER TABLE mt3 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} - ) + node1.query("SYSTEM START MERGES mt3") - partition_id = node1.query("select partitionId('x')").strip() - remove_part_from_disk(node1, "mt3", f"{partition_id}_0_0_0") + for i in range(100): + result = node1.query( + "SELECT count() FROM system.replication_queue WHERE table='mt3'" + ) + assert int(result) <= 2, "Have a lot of entries in queue {}".format( + node1.query("SELECT * FROM system.replication_queue FORMAT Vertical") + ) + if node1.contains_in_log("Cannot create empty part") and node1.contains_in_log( + "DROP/DETACH PARTITION" + ): + break + if node1.contains_in_log( + "Created empty part 8b8f0fede53df97513a9fb4cb19dc1e4_0_0_0 " + ): + break + time.sleep(0.5) + else: + assert False, "Don't have required messages in node1 log" - # other way to detect broken parts - node1.query("CHECK TABLE mt3") + node1.query(f"ALTER TABLE mt3 DROP PARTITION ID '{partition_id}'") - node1.query("SYSTEM START MERGES mt3") - - for i in range(10): - result = node1.query( - "SELECT count() FROM system.replication_queue WHERE table='mt3'" - ) - assert int(result) <= 2, "Have a lot of entries in queue {}".format( - node1.query("SELECT * FROM system.replication_queue FORMAT Vertical") - ) - if node1.contains_in_log("Cannot create empty part") and node1.contains_in_log( - "DROP/DETACH PARTITION" - ): - break - if node1.contains_in_log( - "Created empty part 8b8f0fede53df97513a9fb4cb19dc1e4_0_0_0 " - ): - break - time.sleep(1) - else: - assert False, "Don't have required messages in node1 log" - - node1.query(f"ALTER TABLE mt3 DROP PARTITION ID '{partition_id}'") - - assert_eq_with_retry(node1, "SELECT COUNT() FROM mt3", "0") - assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt3 SYNC") - node2.query("DROP TABLE IF EXISTS mt3 SYNC") + assert_eq_with_retry(node1, "SELECT COUNT() FROM mt3", "0") + assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt3 SYNC") + node2.query("DROP TABLE IF EXISTS mt3 SYNC") From a2e63404b3ffc9a1a9a5e6380918ab4c033e55a2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 12:01:17 +0000 Subject: [PATCH 279/473] Further simplify logic --- cmake/limit_jobs.cmake | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b7cc5cf0967..4d81619aa13 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -45,15 +45,14 @@ endif() message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") -if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () -if (PARALLEL_LINK_JOBS AND (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () - From 355faa4251f7870492e0d019e3ca982bd93f1e90 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 17 Jul 2023 20:08:32 +0800 Subject: [PATCH 280/473] ci fix --- docs/en/interfaces/formats.md | 4 ++-- docs/en/operations/settings/settings-formats.md | 8 ++++---- src/Core/Settings.h | 4 ++-- src/Formats/FormatFactory.cpp | 4 ++-- src/Formats/FormatSettings.h | 4 ++-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 4 ++-- 6 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2207a6f0442..f45c55a9734 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. -- [input_format_csv_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialize_failed) - Set default value to column if the csv input field deserialize failed. Default value - `false`. +- [input_format_csv_allow_check_field_deserialization](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_field_deserialization) - Allow to check whether the csv input field can be successful deserialized. Default value - `false`. +- [input_format_csv_set_default_if_deserialization_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialization_failed) - Set default value to column if the csv input field deserialization failed. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index b74504ac5fd..409ac4bd58a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,15 +969,15 @@ Result a b ``` -### input_format_csv_allow_check_deserialize {#input_format_csv_allow_check_deserialize} +### input_format_csv_allow_check_field_deserialization {#input_format_csv_allow_check_field_deserialization} Allow to use whitespace or tab as field delimiter in CSV strings. Default value: `false`. -### input_format_csv_set_default_if_deserialize_failed {#input_format_csv_set_default_if_deserialize_failed} +### input_format_csv_set_default_if_deserialization_failed {#input_format_csv_set_default_if_deserialization_failed} -Allow to set default value to column if the csv input field deserialize failed +Allow to set default value to column if the csv input field's deserialization failed Default value: `false`. @@ -988,7 +988,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_field_deserialization=true, input_format_csv_set_default_if_deserialization_failed=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5ea355c61c2..9d93ba9ad2c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,8 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_deserialize, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ - M(Bool, input_format_csv_set_default_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ + M(Bool, input_format_csv_allow_check_field_deserialization, false, "Allow to check the csv input field deserialization whether success or not.", 0) \ + M(Bool, input_format_csv_set_default_if_deserialization_failed, false, "All to set column default value if the input field's deserialization failed.", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ac7ceb7e791..b3b9609f9fe 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,8 +73,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.allow_check_deserialize = settings.input_format_csv_allow_check_deserialize; - format_settings.csv.set_default_if_deserialize_failed = settings.input_format_csv_set_default_if_deserialize_failed; + format_settings.csv.allow_check_field_deserialization = settings.input_format_csv_allow_check_field_deserialization; + format_settings.csv.set_default_if_deserialization_failed = settings.input_format_csv_set_default_if_deserialization_failed; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e9de7a51108..4d4eb926992 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,8 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_deserialize=false; - bool set_default_if_deserialize_failed=false; + bool allow_check_field_deserialization=false; + bool set_default_if_deserialization_failed=false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index ada9d78711a..e1be6b21610 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -320,7 +320,7 @@ bool CSVFormatReader::readField( size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_deserialize) + if (format_settings.csv.allow_check_field_deserialization) { std::string field; readCSVField(field, *buf, format_settings.csv); @@ -346,7 +346,7 @@ bool CSVFormatReader::readField( catch (Exception & e) { LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.set_default_if_deserialize_failed) + if (format_settings.csv.set_default_if_deserialization_failed) { // Reset the column and buffer position, then skip the field and set column default value. if (column.size() == col_size + 1) From 71e84656636b0b7a74974d209f3913e0ee1fb5bf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 17 Jul 2023 12:09:36 +0000 Subject: [PATCH 281/473] Automatic style fix --- tests/integration/test_lost_part/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_lost_part/test.py b/tests/integration/test_lost_part/test.py index b83707145f4..382539df7de 100644 --- a/tests/integration/test_lost_part/test.py +++ b/tests/integration/test_lost_part/test.py @@ -278,9 +278,9 @@ def test_lost_last_part(start_cluster): assert int(result) <= 2, "Have a lot of entries in queue {}".format( node1.query("SELECT * FROM system.replication_queue FORMAT Vertical") ) - if node1.contains_in_log("Cannot create empty part") and node1.contains_in_log( - "DROP/DETACH PARTITION" - ): + if node1.contains_in_log( + "Cannot create empty part" + ) and node1.contains_in_log("DROP/DETACH PARTITION"): break if node1.contains_in_log( "Created empty part 8b8f0fede53df97513a9fb4cb19dc1e4_0_0_0 " From 018546a57d4553c44613c11aa3b0eb616461e60c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Jul 2023 17:39:50 +0200 Subject: [PATCH 282/473] Avoid keeping lock Context::getLock() while recalculating access rights of a connected user. --- src/Access/AccessControl.cpp | 53 +++------ src/Access/AccessControl.h | 10 +- src/Access/ContextAccess.cpp | 74 +++++++------ src/Access/ContextAccess.h | 51 +-------- src/Access/ContextAccessParams.cpp | 172 +++++++++++++++++++++++++++++ src/Access/ContextAccessParams.h | 64 +++++++++++ src/Interpreters/Context.cpp | 102 +++++++++-------- src/Interpreters/Context.h | 7 +- 8 files changed, 362 insertions(+), 171 deletions(-) create mode 100644 src/Access/ContextAccessParams.cpp create mode 100644 src/Access/ContextAccessParams.h diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 6179c823b56..41ac3f42ee2 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -72,18 +72,26 @@ public: std::shared_ptr getContextAccess(const ContextAccessParams & params) { - std::lock_guard lock{mutex}; - auto x = cache.get(params); - if (x) { - if ((*x)->tryGetUser()) - return *x; - /// No user, probably the user has been dropped while it was in the cache. - cache.remove(params); + std::lock_guard lock{mutex}; + auto x = cache.get(params); + if (x) + { + if ((*x)->getUserID() && !(*x)->tryGetUser()) + cache.remove(params); /// The user has been dropped while it was in the cache. + else + return *x; + } } + auto res = std::make_shared(access_control, params); res->initialize(); - cache.add(params, res); + + { + std::lock_guard lock{mutex}; + cache.add(params, res); + } + return res; } @@ -713,35 +721,6 @@ int AccessControl::getBcryptWorkfactor() const } -std::shared_ptr AccessControl::getContextAccess( - const UUID & user_id, - const std::vector & current_roles, - bool use_default_roles, - const Settings & settings, - const String & current_database, - const ClientInfo & client_info) const -{ - ContextAccessParams params; - params.user_id = user_id; - params.current_roles.insert(current_roles.begin(), current_roles.end()); - params.use_default_roles = use_default_roles; - params.current_database = current_database; - params.readonly = settings.readonly; - params.allow_ddl = settings.allow_ddl; - params.allow_introspection = settings.allow_introspection_functions; - params.interface = client_info.interface; - params.http_method = client_info.http_method; - params.address = client_info.current_address.host(); - params.quota_key = client_info.quota_key; - - /// Extract the last entry from comma separated list of X-Forwarded-For addresses. - /// Only the last proxy can be trusted (if any). - params.forwarded_address = client_info.getLastForwardedFor(); - - return getContextAccess(params); -} - - std::shared_ptr AccessControl::getContextAccess(const ContextAccessParams & params) const { return context_access_cache->getContextAccess(params); diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 2a8293a49e7..74816090f88 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -25,7 +25,7 @@ namespace Poco namespace DB { class ContextAccess; -struct ContextAccessParams; +class ContextAccessParams; struct User; using UserPtr = std::shared_ptr; class EnabledRoles; @@ -181,14 +181,6 @@ public: void setSettingsConstraintsReplacePrevious(bool enable) { settings_constraints_replace_previous = enable; } bool doesSettingsConstraintsReplacePrevious() const { return settings_constraints_replace_previous; } - std::shared_ptr getContextAccess( - const UUID & user_id, - const std::vector & current_roles, - bool use_default_roles, - const Settings & settings, - const String & current_database, - const ClientInfo & client_info) const; - std::shared_ptr getContextAccess(const ContextAccessParams & params) const; std::shared_ptr getEnabledRoles( diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 9c57853679f..cb8f1a5a48e 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -221,15 +222,15 @@ namespace } -ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_) - : access_control(&access_control_) - , params(params_) +std::shared_ptr ContextAccess::fromContext(const ContextPtr & context) { + return context->getAccess(); } -ContextAccess::ContextAccess(FullAccess) - : is_full_access(true), access(std::make_shared(AccessRights::getFullAccess())), access_with_implicit(access) +ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_) + : access_control(&access_control_) + , params(params_) { } @@ -251,18 +252,31 @@ ContextAccess::~ContextAccess() void ContextAccess::initialize() { - std::lock_guard lock{mutex}; - subscription_for_user_change = access_control->subscribeForChanges( - *params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) - { - auto ptr = weak_ptr.lock(); - if (!ptr) - return; - UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; - std::lock_guard lock2{ptr->mutex}; - ptr->setUser(changed_user); - }); - setUser(access_control->read(*params.user_id)); + std::lock_guard lock{mutex}; + + if (params.full_access) + { + access = std::make_shared(AccessRights::getFullAccess()); + access_with_implicit = access; + return; + } + + if (!params.user_id) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug"); + + subscription_for_user_change = access_control->subscribeForChanges( + *params.user_id, + [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) + { + auto ptr = weak_ptr.lock(); + if (!ptr) + return; + UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; + std::lock_guard lock2{ptr->mutex}; + ptr->setUser(changed_user); + }); + + setUser(access_control->read(*params.user_id)); } @@ -294,10 +308,10 @@ void ContextAccess::setUser(const UserPtr & user_) const current_roles = user->granted_roles.findGranted(user->default_roles); current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(user->default_roles); } - else + else if (params.current_roles) { - current_roles = user->granted_roles.findGranted(params.current_roles); - current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(params.current_roles); + current_roles = user->granted_roles.findGranted(*params.current_roles); + current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(*params.current_roles); } subscription_for_roles_changes.reset(); @@ -316,12 +330,16 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & { assert(roles_info_); roles_info = roles_info_; + enabled_row_policies = access_control->getEnabledRowPolicies( *params.user_id, roles_info->enabled_roles); + enabled_quota = access_control->getEnabledQuota( *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); + enabled_settings = access_control->getEnabledSettings( *params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); + calculateAccessRights(); } @@ -417,14 +435,6 @@ std::optional ContextAccess::getQuotaUsage() const } -std::shared_ptr ContextAccess::getFullAccess() -{ - static const std::shared_ptr res = - [] { return std::shared_ptr(new ContextAccess{kFullAccess}); }(); - return res; -} - - SettingsChanges ContextAccess::getDefaultSettings() const { std::lock_guard lock{mutex}; @@ -478,7 +488,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg throw Exception(ErrorCodes::UNKNOWN_USER, "{}: User has been dropped", getUserName()); } - if (is_full_access) + if (params.full_access) return true; auto access_granted = [&] @@ -706,7 +716,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const return false; }; - if (is_full_access) + if (params.full_access) return true; if (user_was_dropped) @@ -806,7 +816,7 @@ void ContextAccess::checkAdminOption(const std::vector & role_ids, const s void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const { - if (is_full_access) + if (params.full_access) return; auto current_user = getUser(); @@ -816,7 +826,7 @@ void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccess void ContextAccess::checkGranteesAreAllowed(const std::vector & grantee_ids) const { - if (is_full_access) + if (params.full_access) return; auto current_user = getUser(); diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 60bad0118fc..4c96ef5c11f 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -30,47 +31,18 @@ class AccessControl; class IAST; struct IAccessEntity; using ASTPtr = std::shared_ptr; - - -struct ContextAccessParams -{ - std::optional user_id; - boost::container::flat_set current_roles; - bool use_default_roles = false; - UInt64 readonly = 0; - bool allow_ddl = false; - bool allow_introspection = false; - String current_database; - ClientInfo::Interface interface = ClientInfo::Interface::TCP; - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - Poco::Net::IPAddress address; - String forwarded_address; - String quota_key; - - auto toTuple() const - { - return std::tie( - user_id, current_roles, use_default_roles, readonly, allow_ddl, allow_introspection, - current_database, interface, http_method, address, forwarded_address, quota_key); - } - - friend bool operator ==(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() == rhs.toTuple(); } - friend bool operator !=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs == rhs); } - friend bool operator <(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() < rhs.toTuple(); } - friend bool operator >(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return rhs < lhs; } - friend bool operator <=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(rhs < lhs); } - friend bool operator >=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs < rhs); } -}; +class Context; +using ContextPtr = std::shared_ptr; class ContextAccess : public std::enable_shared_from_this { public: + static std::shared_ptr fromContext(const ContextPtr & context); + using Params = ContextAccessParams; const Params & getParams() const { return params; } - ContextAccess(const AccessControl & access_control_, const Params & params_); - /// Returns the current user. Throws if user is nullptr. UserPtr getUser() const; /// Same as above, but can return nullptr. @@ -161,22 +133,12 @@ public: /// Checks if grantees are allowed for the current user, throws an exception if not. void checkGranteesAreAllowed(const std::vector & grantee_ids) const; - /// Makes an instance of ContextAccess which provides full access to everything - /// without any limitations. This is used for the global context. - static std::shared_ptr getFullAccess(); - + ContextAccess(const AccessControl & access_control_, const Params & params_); ~ContextAccess(); private: friend class AccessControl; - struct FullAccess {}; - static const FullAccess kFullAccess; - - /// Makes an instance of ContextAccess which provides full access to everything - /// without any limitations. This is used for the global context. - explicit ContextAccess(FullAccess); - void initialize(); void setUser(const UserPtr & user_) const TSA_REQUIRES(mutex); void setRolesInfo(const std::shared_ptr & roles_info_) const TSA_REQUIRES(mutex); @@ -223,7 +185,6 @@ private: const AccessControl * access_control = nullptr; const Params params; - const bool is_full_access = false; mutable std::atomic user_was_dropped = false; mutable std::atomic trace_log = nullptr; diff --git a/src/Access/ContextAccessParams.cpp b/src/Access/ContextAccessParams.cpp new file mode 100644 index 00000000000..7963e83dddf --- /dev/null +++ b/src/Access/ContextAccessParams.cpp @@ -0,0 +1,172 @@ +#include +#include +#include + + +namespace DB +{ + +ContextAccessParams::ContextAccessParams( + const std::optional user_id_, + bool full_access_, + bool use_default_roles_, + const std::shared_ptr> & current_roles_, + const Settings & settings_, + const String & current_database_, + const ClientInfo & client_info_) + : user_id(user_id_) + , full_access(full_access_) + , use_default_roles(use_default_roles_) + , current_roles(current_roles_) + , readonly(settings_.readonly) + , allow_ddl(settings_.allow_ddl) + , allow_introspection(settings_.allow_introspection_functions) + , current_database(current_database_) + , interface(client_info_.interface) + , http_method(client_info_.http_method) + , address(client_info_.current_address.host()) + , forwarded_address(client_info_.getLastForwardedFor()) + , quota_key(client_info_.quota_key) +{ +} + +String ContextAccessParams::toString() const +{ + WriteBufferFromOwnString out; + auto separator = [&] { return out.stringView().empty() ? "" : ", "; }; + if (user_id) + out << separator() << "user_id = " << *user_id; + if (full_access) + out << separator() << "full_access = " << full_access; + if (use_default_roles) + out << separator() << "use_default_roles = " << use_default_roles; + if (current_roles && !current_roles->empty()) + { + out << separator() << "current_roles = ["; + for (size_t i = 0; i != current_roles->size(); ++i) + { + if (i) + out << ", "; + out << (*current_roles)[i]; + } + out << "]"; + } + if (readonly) + out << separator() << "readonly = " << readonly; + if (allow_ddl) + out << separator() << "allow_ddl = " << allow_ddl; + if (allow_introspection) + out << separator() << "allow_introspection = " << allow_introspection; + if (!current_database.empty()) + out << separator() << "current_database = " << current_database; + out << separator() << "interface = " << magic_enum::enum_name(interface); + if (http_method != ClientInfo::HTTPMethod::UNKNOWN) + out << separator() << "http_method = " << magic_enum::enum_name(http_method); + if (!address.isWildcard()) + out << separator() << "address = " << address.toString(); + if (!forwarded_address.empty()) + out << separator() << "forwarded_address = " << forwarded_address; + if (!quota_key.empty()) + out << separator() << "quota_key = " << quota_key; + return out.str(); +} + +bool operator ==(const ContextAccessParams & left, const ContextAccessParams & right) +{ + auto check_equals = [](const auto & x, const auto & y) + { + if constexpr (::detail::is_shared_ptr_v>) + { + if (!x) + return !y; + else if (!y) + return false; + else + return *x == *y; + } + else + { + return x == y; + } + }; + + #define CONTEXT_ACCESS_PARAMS_EQUALS(name) \ + if (!check_equals(left.name, right.name)) \ + return false; + + CONTEXT_ACCESS_PARAMS_EQUALS(user_id) + CONTEXT_ACCESS_PARAMS_EQUALS(full_access) + CONTEXT_ACCESS_PARAMS_EQUALS(use_default_roles) + CONTEXT_ACCESS_PARAMS_EQUALS(current_roles) + CONTEXT_ACCESS_PARAMS_EQUALS(readonly) + CONTEXT_ACCESS_PARAMS_EQUALS(allow_ddl) + CONTEXT_ACCESS_PARAMS_EQUALS(allow_introspection) + CONTEXT_ACCESS_PARAMS_EQUALS(current_database) + CONTEXT_ACCESS_PARAMS_EQUALS(interface) + CONTEXT_ACCESS_PARAMS_EQUALS(http_method) + CONTEXT_ACCESS_PARAMS_EQUALS(address) + CONTEXT_ACCESS_PARAMS_EQUALS(forwarded_address) + CONTEXT_ACCESS_PARAMS_EQUALS(quota_key) + + #undef CONTEXT_ACCESS_PARAMS_EQUALS + + return true; /// All fields are equal, operator == must return true. +} + +bool operator <(const ContextAccessParams & left, const ContextAccessParams & right) +{ + auto check_less = [](const auto & x, const auto & y) + { + if constexpr (::detail::is_shared_ptr_v>) + { + if (!x) + return y ? -1 : 0; + else if (!y) + return 1; + else if (*x == *y) + return 0; + else if (*x < *y) + return -1; + else + return 1; + } + else + { + if (x == y) + return 0; + else if (x < y) + return -1; + else + return 1; + } + }; + + #define CONTEXT_ACCESS_PARAMS_LESS(name) \ + if (auto cmp = check_less(left.name, right.name); cmp != 0) \ + return cmp < 0; + + CONTEXT_ACCESS_PARAMS_LESS(user_id) + CONTEXT_ACCESS_PARAMS_LESS(full_access) + CONTEXT_ACCESS_PARAMS_LESS(use_default_roles) + CONTEXT_ACCESS_PARAMS_LESS(current_roles) + CONTEXT_ACCESS_PARAMS_LESS(readonly) + CONTEXT_ACCESS_PARAMS_LESS(allow_ddl) + CONTEXT_ACCESS_PARAMS_LESS(allow_introspection) + CONTEXT_ACCESS_PARAMS_LESS(current_database) + CONTEXT_ACCESS_PARAMS_LESS(interface) + CONTEXT_ACCESS_PARAMS_LESS(http_method) + CONTEXT_ACCESS_PARAMS_LESS(address) + CONTEXT_ACCESS_PARAMS_LESS(forwarded_address) + CONTEXT_ACCESS_PARAMS_LESS(quota_key) + + #undef CONTEXT_ACCESS_PARAMS_LESS + + return false; /// All fields are equal, operator < must return false. +} + +bool ContextAccessParams::dependsOnSettingName(std::string_view setting_name) +{ + return (setting_name == "readonly") || (setting_name == "allow_ddl") || (setting_name == "allow_introspection_functions"); +} + +} diff --git a/src/Access/ContextAccessParams.h b/src/Access/ContextAccessParams.h new file mode 100644 index 00000000000..740ec997964 --- /dev/null +++ b/src/Access/ContextAccessParams.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +struct Settings; + +/// Parameters which are used to calculate access rights and some related stuff like roles or constraints. +class ContextAccessParams +{ +public: + ContextAccessParams( + const std::optional user_id_, + bool full_access_, + bool use_default_roles_, + const std::shared_ptr> & current_roles_, + const Settings & settings_, + const String & current_database_, + const ClientInfo & client_info_); + + const std::optional user_id; + + /// Full access to everything without any limitations. + /// This is used for the global context. + const bool full_access; + + const bool use_default_roles; + const std::shared_ptr> current_roles; + + const UInt64 readonly; + const bool allow_ddl; + const bool allow_introspection; + + const String current_database; + + const ClientInfo::Interface interface; + const ClientInfo::HTTPMethod http_method; + const Poco::Net::IPAddress address; + + /// The last entry from comma separated list of X-Forwarded-For addresses. + /// Only the last proxy can be trusted (if any). + const String forwarded_address; + + const String quota_key; + + /// Outputs `ContextAccessParams` to string for logging. + String toString() const; + + friend bool operator <(const ContextAccessParams & left, const ContextAccessParams & right); + friend bool operator ==(const ContextAccessParams & left, const ContextAccessParams & right); + friend bool operator !=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(left == right); } + friend bool operator >(const ContextAccessParams & left, const ContextAccessParams & right) { return right < left; } + friend bool operator <=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(right < left); } + friend bool operator >=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(left < right); } + + static bool dependsOnSettingName(std::string_view setting_name); +}; + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5019933c2af..abc33c1b8d4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1063,8 +1063,16 @@ void Context::setUser(const UUID & user_id_) user_id = user_id_; - access = getAccessControl().getContextAccess( - user_id_, /* current_roles = */ {}, /* use_default_roles = */ true, settings, current_database, client_info); + ContextAccessParams params{ + user_id, + /* full_access= */ false, + /* use_default_roles = */ true, + /* current_roles = */ nullptr, + settings, + current_database, + client_info}; + + access = getAccessControl().getContextAccess(params); auto user = access->getUser(); @@ -1108,7 +1116,7 @@ void Context::setCurrentRoles(const std::vector & current_roles_) if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty()) return; current_roles = std::make_shared>(current_roles_); - calculateAccessRights(); + need_recalculate_access = true; } void Context::setCurrentRolesDefault() @@ -1133,20 +1141,6 @@ std::shared_ptr Context::getRolesInfo() const } -void Context::calculateAccessRights() -{ - auto lock = getLock(); - if (user_id) - access = getAccessControl().getContextAccess( - *user_id, - current_roles ? *current_roles : std::vector{}, - /* use_default_roles = */ false, - settings, - current_database, - client_info); -} - - template void Context::checkAccessImpl(const Args &... args) const { @@ -1166,11 +1160,50 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } - std::shared_ptr Context::getAccess() const { - auto lock = getLock(); - return access ? access : ContextAccess::getFullAccess(); + /// A helper function to collect parameters for calculating access rights, called with Context::getLock() acquired. + auto get_params = [this]() + { + /// If setUserID() was never called then this must be the global context with the full access. + bool full_access = !user_id; + + return ContextAccessParams{user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info}; + }; + + /// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights. + std::optional params; + + { + auto lock = getLock(); + if (access && !need_recalculate_access) + return access; /// No need to recalculate access rights. + + params.emplace(get_params()); + + if (access && (access->getParams() == *params)) + { + need_recalculate_access = false; + return access; /// No need to recalculate access rights. + } + } + + /// Calculate new access rights according to the collected parameters. + /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. + auto res = getAccessControl().getContextAccess(*params); + + { + /// If the parameters of access rights were not changed while we were calculated them + /// then we store the new access rights in the Context to allow reusing it later. + auto lock = getLock(); + if (get_params() == *params) + { + access = res; + need_recalculate_access = false; + } + } + + return res; } RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const @@ -1700,27 +1733,8 @@ Settings Context::getSettings() const void Context::setSettings(const Settings & settings_) { auto lock = getLock(); - const auto old_readonly = settings.readonly; - const auto old_allow_ddl = settings.allow_ddl; - const auto old_allow_introspection_functions = settings.allow_introspection_functions; - const auto old_display_secrets = settings.format_display_secrets_in_show_and_select; - settings = settings_; - - if ((settings.readonly != old_readonly) - || (settings.allow_ddl != old_allow_ddl) - || (settings.allow_introspection_functions != old_allow_introspection_functions) - || (settings.format_display_secrets_in_show_and_select != old_display_secrets)) - calculateAccessRights(); -} - -void Context::recalculateAccessRightsIfNeeded(std::string_view name) -{ - if (name == "readonly" - || name == "allow_ddl" - || name == "allow_introspection_functions" - || name == "format_display_secrets_in_show_and_select") - calculateAccessRights(); + need_recalculate_access = true; } void Context::setSetting(std::string_view name, const String & value) @@ -1732,7 +1746,8 @@ void Context::setSetting(std::string_view name, const String & value) return; } settings.set(name, value); - recalculateAccessRightsIfNeeded(name); + if (ContextAccessParams::dependsOnSettingName(name)) + need_recalculate_access = true; } void Context::setSetting(std::string_view name, const Field & value) @@ -1744,7 +1759,8 @@ void Context::setSetting(std::string_view name, const Field & value) return; } settings.set(name, value); - recalculateAccessRightsIfNeeded(name); + if (ContextAccessParams::dependsOnSettingName(name)) + need_recalculate_access = true; } void Context::applySettingChange(const SettingChange & change) @@ -1853,7 +1869,7 @@ void Context::setCurrentDatabase(const String & name) DatabaseCatalog::instance().assertDatabaseExists(name); auto lock = getLock(); current_database = name; - calculateAccessRights(); + need_recalculate_access = true; } void Context::setCurrentQueryId(const String & query_id) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6cbb0e58911..2c32ad28d01 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -248,7 +248,8 @@ private: std::optional user_id; std::shared_ptr> current_roles; std::shared_ptr settings_constraints_and_current_profiles; - std::shared_ptr access; + mutable std::shared_ptr access; + mutable bool need_recalculate_access = true; std::shared_ptr row_policies_of_initial_user; String current_database; Settings settings; /// Setting for query execution. @@ -1149,10 +1150,6 @@ private: void initGlobal(); - /// Compute and set actual user settings, client_info.current_user should be set - void calculateAccessRights(); - void recalculateAccessRightsIfNeeded(std::string_view setting_name); - template void checkAccessImpl(const Args &... args) const; From 0e4b75a282f14d812763a43cb3519c94beeb138c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Jul 2023 19:58:38 +0200 Subject: [PATCH 283/473] Avoid keeping lock Context::getLock() while calculating access rights when a user logs in. --- src/Access/ContextAccess.cpp | 35 +++++---- src/Access/ContextAccess.h | 6 +- src/Access/ContextAccessParams.cpp | 5 ++ src/Access/ContextAccessParams.h | 3 + src/Interpreters/Context.cpp | 115 +++++++++++++++++------------ src/Interpreters/Context.h | 23 +++--- src/Interpreters/Session.cpp | 3 - 7 files changed, 107 insertions(+), 83 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index cb8f1a5a48e..51bb7794735 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -240,6 +240,7 @@ ContextAccess::~ContextAccess() enabled_settings.reset(); enabled_quota.reset(); enabled_row_policies.reset(); + row_policies_of_initial_user.reset(); access_with_implicit.reset(); access.reset(); roles_info.reset(); @@ -264,6 +265,12 @@ void ContextAccess::initialize() if (!params.user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug"); + if (!params.initial_user.empty()) + { + if (auto initial_user_id = access_control->find(params.initial_user)) + row_policies_of_initial_user = access_control->tryGetDefaultRowPolicies(*initial_user_id); + } + subscription_for_user_change = access_control->subscribeForChanges( *params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) @@ -331,8 +338,7 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & assert(roles_info_); roles_info = roles_info_; - enabled_row_policies = access_control->getEnabledRowPolicies( - *params.user_id, roles_info->enabled_roles); + enabled_row_policies = access_control->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles); enabled_quota = access_control->getEnabledQuota( *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); @@ -399,21 +405,24 @@ std::shared_ptr ContextAccess::getRolesInfo() const return no_roles; } -std::shared_ptr ContextAccess::getEnabledRowPolicies() const +RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { std::lock_guard lock{mutex}; - if (enabled_row_policies) - return enabled_row_policies; - static const auto no_row_policies = std::make_shared(); - return no_row_policies; -} -RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter) const -{ - std::lock_guard lock{mutex}; + RowPolicyFilterPtr filter; if (enabled_row_policies) - return enabled_row_policies->getFilter(database, table_name, filter_type, combine_with_filter); - return combine_with_filter; + filter = enabled_row_policies->getFilter(database, table_name, filter_type); + + if (row_policies_of_initial_user) + { + /// Find and set extra row policies to be used based on `client_info.initial_user`, if the initial user exists. + /// TODO: we need a better solution here. It seems we should pass the initial row policy + /// because a shard is allowed to not have the initial user or it might be another user + /// with the same name. + filter = row_policies_of_initial_user->getFilter(database, table_name, filter_type, filter); + } + + return filter; } std::shared_ptr ContextAccess::getQuota() const diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 4c96ef5c11f..4bd67f8881b 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -53,12 +53,9 @@ public: /// Returns information about current and enabled roles. std::shared_ptr getRolesInfo() const; - /// Returns information about enabled row policies. - std::shared_ptr getEnabledRowPolicies() const; - /// Returns the row policy filter for a specified table. /// The function returns nullptr if there is no filter to apply. - RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter = {}) const; + RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; /// Returns the quota to track resource consumption. std::shared_ptr getQuota() const; @@ -198,6 +195,7 @@ private: mutable std::shared_ptr access TSA_GUARDED_BY(mutex); mutable std::shared_ptr access_with_implicit TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_row_policies TSA_GUARDED_BY(mutex); + mutable std::shared_ptr row_policies_of_initial_user TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_quota TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_settings TSA_GUARDED_BY(mutex); diff --git a/src/Access/ContextAccessParams.cpp b/src/Access/ContextAccessParams.cpp index 7963e83dddf..ec839a37b1a 100644 --- a/src/Access/ContextAccessParams.cpp +++ b/src/Access/ContextAccessParams.cpp @@ -27,6 +27,7 @@ ContextAccessParams::ContextAccessParams( , address(client_info_.current_address.host()) , forwarded_address(client_info_.getLastForwardedFor()) , quota_key(client_info_.quota_key) + , initial_user((client_info_.initial_user != client_info_.current_user) ? client_info_.initial_user : "") { } @@ -68,6 +69,8 @@ String ContextAccessParams::toString() const out << separator() << "forwarded_address = " << forwarded_address; if (!quota_key.empty()) out << separator() << "quota_key = " << quota_key; + if (!initial_user.empty()) + out << separator() << "initial_user = " << initial_user; return out.str(); } @@ -107,6 +110,7 @@ bool operator ==(const ContextAccessParams & left, const ContextAccessParams & r CONTEXT_ACCESS_PARAMS_EQUALS(address) CONTEXT_ACCESS_PARAMS_EQUALS(forwarded_address) CONTEXT_ACCESS_PARAMS_EQUALS(quota_key) + CONTEXT_ACCESS_PARAMS_EQUALS(initial_user) #undef CONTEXT_ACCESS_PARAMS_EQUALS @@ -158,6 +162,7 @@ bool operator <(const ContextAccessParams & left, const ContextAccessParams & ri CONTEXT_ACCESS_PARAMS_LESS(address) CONTEXT_ACCESS_PARAMS_LESS(forwarded_address) CONTEXT_ACCESS_PARAMS_LESS(quota_key) + CONTEXT_ACCESS_PARAMS_LESS(initial_user) #undef CONTEXT_ACCESS_PARAMS_LESS diff --git a/src/Access/ContextAccessParams.h b/src/Access/ContextAccessParams.h index 740ec997964..8b68fa44ed4 100644 --- a/src/Access/ContextAccessParams.h +++ b/src/Access/ContextAccessParams.h @@ -48,6 +48,9 @@ public: const String quota_key; + /// Initial user is used to combine row policies with. + const String initial_user; + /// Outputs `ContextAccessParams` to string for logging. String toString() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index abc33c1b8d4..5fae9374705 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1057,33 +1057,54 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_) +void Context::setUser(const UUID & user_id_, bool set_current_profiles_, bool set_current_roles_, bool set_current_database_) { + /// Prepare lists of user's profiles, constraints, settings, roles. + + std::shared_ptr user; + std::shared_ptr temp_access; + if (set_current_profiles_ || set_current_roles_ || set_current_database_) + { + std::optional params; + { + auto lock = getLock(); + params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info}); + } + /// `temp_access` is used here only to extract information about the user, not to actually check access. + /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. + temp_access = getAccessControl().getContextAccess(*params); + user = temp_access->getUser(); + } + + std::shared_ptr profiles; + if (set_current_profiles_) + profiles = temp_access->getDefaultProfileInfo(); + + std::optional> roles; + if (set_current_roles_) + roles = user->granted_roles.findGranted(user->default_roles); + + String database; + if (set_current_database_) + database = user->default_database; + + /// Apply user's profiles, constraints, settings, roles. auto lock = getLock(); - user_id = user_id_; + setUserID(user_id_); - ContextAccessParams params{ - user_id, - /* full_access= */ false, - /* use_default_roles = */ true, - /* current_roles = */ nullptr, - settings, - current_database, - client_info}; + if (profiles) + { + /// A profile can specify a value and a readonly constraint for same setting at the same time, + /// so we shouldn't check constraints here. + setCurrentProfiles(*profiles, /* check_constraints= */ false); + } - access = getAccessControl().getContextAccess(params); + if (roles) + setCurrentRoles(*roles); - auto user = access->getUser(); - - current_roles = std::make_shared>(user->granted_roles.findGranted(user->default_roles)); - - auto default_profile_info = access->getDefaultProfileInfo(); - settings_constraints_and_current_profiles = default_profile_info->getConstraintsAndProfileIDs(); - applySettingsChanges(default_profile_info->settings); - - if (!user->default_database.empty()) - setCurrentDatabase(user->default_database); + if (!database.empty()) + setCurrentDatabase(database); } std::shared_ptr Context::getUser() const @@ -1096,6 +1117,13 @@ String Context::getUserName() const return getAccess()->getUserName(); } +void Context::setUserID(const UUID & user_id_) +{ + auto lock = getLock(); + user_id = user_id_; + need_recalculate_access = true; +} + std::optional Context::getUserID() const { auto lock = getLock(); @@ -1113,9 +1141,10 @@ void Context::setQuotaKey(String quota_key_) void Context::setCurrentRoles(const std::vector & current_roles_) { auto lock = getLock(); - if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty()) - return; - current_roles = std::make_shared>(current_roles_); + if (current_roles_.empty()) + current_roles = nullptr; + else + current_roles = std::make_shared>(current_roles_); need_recalculate_access = true; } @@ -1208,23 +1237,7 @@ std::shared_ptr Context::getAccess() const RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { - auto lock = getLock(); - RowPolicyFilterPtr row_filter_of_initial_user; - if (row_policies_of_initial_user) - row_filter_of_initial_user = row_policies_of_initial_user->getFilter(database, table_name, filter_type); - return getAccess()->getRowPolicyFilter(database, table_name, filter_type, row_filter_of_initial_user); -} - -void Context::enableRowPoliciesOfInitialUser() -{ - auto lock = getLock(); - row_policies_of_initial_user = nullptr; - if (client_info.initial_user == client_info.current_user) - return; - auto initial_user_id = getAccessControl().find(client_info.initial_user); - if (!initial_user_id) - return; - row_policies_of_initial_user = getAccessControl().tryGetDefaultRowPolicies(*initial_user_id); + return getAccess()->getRowPolicyFilter(database, table_name, filter_type); } @@ -1240,13 +1253,12 @@ std::optional Context::getQuotaUsage() const } -void Context::setCurrentProfile(const String & profile_name) +void Context::setCurrentProfile(const String & profile_name, bool check_constraints) { - auto lock = getLock(); try { UUID profile_id = getAccessControl().getID(profile_name); - setCurrentProfile(profile_id); + setCurrentProfile(profile_id, check_constraints); } catch (Exception & e) { @@ -1255,15 +1267,20 @@ void Context::setCurrentProfile(const String & profile_name) } } -void Context::setCurrentProfile(const UUID & profile_id) +void Context::setCurrentProfile(const UUID & profile_id, bool check_constraints) { - auto lock = getLock(); auto profile_info = getAccessControl().getSettingsProfileInfo(profile_id); - checkSettingsConstraints(profile_info->settings); - applySettingsChanges(profile_info->settings); - settings_constraints_and_current_profiles = profile_info->getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); + setCurrentProfiles(*profile_info, check_constraints); } +void Context::setCurrentProfiles(const SettingsProfilesInfo & profiles_info, bool check_constraints) +{ + auto lock = getLock(); + if (check_constraints) + checkSettingsConstraints(profiles_info.settings); + applySettingsChanges(profiles_info.settings); + settings_constraints_and_current_profiles = profiles_info.getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); +} std::vector Context::getCurrentProfiles() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2c32ad28d01..172f3818dfd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -50,8 +50,8 @@ struct ContextSharedPart; class ContextAccess; struct User; using UserPtr = std::shared_ptr; +struct SettingsProfilesInfo; struct EnabledRolesInfo; -class EnabledRowPolicies; struct RowPolicyFilter; using RowPolicyFilterPtr = std::shared_ptr; class EnabledQuota; @@ -250,7 +250,6 @@ private: std::shared_ptr settings_constraints_and_current_profiles; mutable std::shared_ptr access; mutable bool need_recalculate_access = true; - std::shared_ptr row_policies_of_initial_user; String current_database; Settings settings; /// Setting for query execution. @@ -530,12 +529,14 @@ public: /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_); - + void setUser(const UUID & user_id_, bool set_current_profiles_ = true, bool set_current_roles_ = true, bool set_current_database_ = true); UserPtr getUser() const; - String getUserName() const; + + void setUserID(const UUID & user_id_); std::optional getUserID() const; + String getUserName() const; + void setQuotaKey(String quota_key_); void setCurrentRoles(const std::vector & current_roles_); @@ -544,8 +545,9 @@ public: boost::container::flat_set getEnabledRoles() const; std::shared_ptr getRolesInfo() const; - void setCurrentProfile(const String & profile_name); - void setCurrentProfile(const UUID & profile_id); + void setCurrentProfile(const String & profile_name, bool check_constraints = true); + void setCurrentProfile(const UUID & profile_id, bool check_constraints = true); + void setCurrentProfiles(const SettingsProfilesInfo & profiles_info, bool check_constraints = true); std::vector getCurrentProfiles() const; std::vector getEnabledProfiles() const; @@ -568,13 +570,6 @@ public: RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; - /// Finds and sets extra row policies to be used based on `client_info.initial_user`, - /// if the initial user exists. - /// TODO: we need a better solution here. It seems we should pass the initial row policy - /// because a shard is allowed to not have the initial user or it might be another user - /// with the same name. - void enableRowPoliciesOfInitialUser(); - std::shared_ptr getQuota() const; std::optional getQuotaUsage() const; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 64f7b4fc934..8571f20b91e 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -515,9 +515,6 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t res_client_info.initial_address = res_client_info.current_address; } - /// Sets that row policies of the initial user should be used too. - query_context->enableRowPoliciesOfInitialUser(); - /// Set user information for the new context: current profiles, roles, access rights. if (user_id && !query_context->getAccess()->tryGetUser()) query_context->setUser(*user_id); From 4da0782e553f90e2cdc742ad51ec30d1786dd422 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Mon, 17 Jul 2023 15:01:06 +0200 Subject: [PATCH 284/473] MaterializedMySQL: Add tests to parse db and table names from DDL --- .../MySQL/MaterializedMySQLSyncThread.cpp | 66 +------ .../gtest_try_parse_table_id_from_ddl.cpp | 185 ++++++++++++++++++ .../MySQL/tryParseTableIDFromDDL.cpp | 44 +++++ src/Databases/MySQL/tryParseTableIDFromDDL.h | 11 ++ 4 files changed, 245 insertions(+), 61 deletions(-) create mode 100644 src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp create mode 100644 src/Databases/MySQL/tryParseTableIDFromDDL.cpp create mode 100644 src/Databases/MySQL/tryParseTableIDFromDDL.h diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index a01ab2a15a8..780619ff079 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -3,6 +3,7 @@ #if USE_MYSQL #include +#include #include #include #include @@ -151,61 +152,6 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S } } -static std::tuple tryExtractTableNameFromDDL(const String & ddl) -{ - String table_name; - String database_name; - if (ddl.empty()) return std::make_tuple(database_name, table_name); - - bool parse_failed = false; - Tokens tokens(ddl.data(), ddl.data() + ddl.size()); - IParser::Pos pos(tokens, 0); - Expected expected; - ASTPtr res; - ASTPtr table; - if (ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos, expected) || ParserKeyword("CREATE TABLE").ignore(pos, expected)) - { - ParserKeyword("IF NOT EXISTS").ignore(pos, expected); - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("ALTER TABLE").ignore(pos, expected)) - { - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("DROP TABLE").ignore(pos, expected) || ParserKeyword("DROP TEMPORARY TABLE").ignore(pos, expected)) - { - ParserKeyword("IF EXISTS").ignore(pos, expected); - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("TRUNCATE").ignore(pos, expected)) - { - ParserKeyword("TABLE").ignore(pos, expected); - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("RENAME TABLE").ignore(pos, expected)) - { - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else - { - parse_failed = true; - } - if (!parse_failed) - { - if (auto table_id = table->as()->getTableId()) - { - database_name = table_id.database_name; - table_name = table_id.table_name; - } - } - return std::make_tuple(database_name, table_name); -} - MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( ContextPtr context_, const String & database_name_, @@ -868,14 +814,12 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even String query = query_event.query; if (!materialized_tables_list.empty()) { - auto [ddl_database_name, ddl_table_name] = tryExtractTableNameFromDDL(query_event.query); - - if (!ddl_table_name.empty()) + auto table_id = tryParseTableIDFromDDL(query, query_event.schema); + if (!table_id.table_name.empty()) { - ddl_database_name = ddl_database_name.empty() ? query_event.schema: ddl_database_name; - if (ddl_database_name != mysql_database_name || !materialized_tables_list.contains(ddl_table_name)) + if (table_id.database_name != mysql_database_name || !materialized_tables_list.contains(table_id.table_name)) { - LOG_DEBUG(log, "Skip MySQL DDL: \n {}", query_event.query); + LOG_DEBUG(log, "Skip MySQL DDL for {}.{}:\n{}", table_id.database_name, table_id.table_name, query); return; } } diff --git a/src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp b/src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp new file mode 100644 index 00000000000..5b60262930f --- /dev/null +++ b/src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp @@ -0,0 +1,185 @@ +#include "config.h" + +#include + +#include + +using namespace DB; + +struct ParseTableIDFromDDLTestCase +{ + String query; + String database_name; + String table_name; + + ParseTableIDFromDDLTestCase( + const String & query_, + const String & database_name_, + const String & table_name_) + : query(query_) + , database_name(database_name_) + , table_name(table_name_) + { + } +}; + +std::ostream & operator<<(std::ostream & ostr, const ParseTableIDFromDDLTestCase & test_case) +{ + return ostr << '"' << test_case.query << "\" extracts `" << test_case.database_name << "`.`" << test_case.table_name << "`"; +} + +class ParseTableIDFromDDLTest : public ::testing::TestWithParam +{ +}; + +TEST_P(ParseTableIDFromDDLTest, parse) +{ + const auto & [query, expected_database_name, expected_table_name] = GetParam(); + auto table_id = tryParseTableIDFromDDL(query, "default"); + EXPECT_EQ(expected_database_name, table_id.database_name); + EXPECT_EQ(expected_table_name, table_id.table_name); +} + +INSTANTIATE_TEST_SUITE_P(MaterializedMySQL, ParseTableIDFromDDLTest, ::testing::ValuesIn(std::initializer_list{ + { + "SELECT * FROM db.table", + "", + "" + }, + { + "CREATE TEMPORARY TABLE db.table", + "db", + "table" + }, + { + "CREATE TEMPORARY TABLE IF NOT EXISTS db.table", + "db", + "table" + }, + { + "CREATE TEMPORARY TABLE table", + "default", + "table" + }, + { + "CREATE TEMPORARY TABLE IF NOT EXISTS table", + "default", + "table" + }, + { + "CREATE TABLE db.table", + "db", + "table" + }, + { + "CREATE TABLE IF NOT EXISTS db.table", + "db", + "table" + }, + { + "CREATE TABLE table", + "default", + "table" + }, + { + "CREATE TABLE IF NOT EXISTS table", + "default", + "table" + }, + { + "ALTER TABLE db.table", + "db", + "table" + }, + { + "ALTER TABLE table", + "default", + "table" + }, + { + "DROP TABLE db.table", + "db", + "table" + }, + { + "DROP TABLE IF EXISTS db.table", + "db", + "table" + }, + { + "DROP TABLE table", + "default", + "table" + }, + { + "DROP TABLE IF EXISTS table", + "default", + "table" + }, + { + "DROP TEMPORARY TABLE db.table", + "db", + "table" + }, + { + "DROP TEMPORARY TABLE IF EXISTS db.table", + "db", + "table" + }, + { + "DROP TEMPORARY TABLE table", + "default", + "table" + }, + { + "DROP TEMPORARY TABLE IF EXISTS table", + "default", + "table" + }, + { + "TRUNCATE db.table", + "db", + "table" + }, + { + "TRUNCATE TABLE db.table", + "db", + "table" + }, + { + "TRUNCATE table1", + "default", + "table1" + }, + { + "TRUNCATE TABLE table", + "default", + "table" + }, + { + "RENAME TABLE db.table", + "db", + "table" + }, + { + "RENAME TABLE table", + "default", + "table" + }, + { + "DROP DATABASE db", + "", + "" + }, + { + "DROP DATA`BASE db", + "", + "" + }, + { + "NOT A SQL", + "", + "" + }, + +})); diff --git a/src/Databases/MySQL/tryParseTableIDFromDDL.cpp b/src/Databases/MySQL/tryParseTableIDFromDDL.cpp new file mode 100644 index 00000000000..a01eb311450 --- /dev/null +++ b/src/Databases/MySQL/tryParseTableIDFromDDL.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + +namespace DB +{ + +StorageID tryParseTableIDFromDDL(const String & query, const String & default_database_name) +{ + bool is_ddl = false; + Tokens tokens(query.data(), query.data() + query.size()); + IParser::Pos pos(tokens, 0); + Expected expected; + if (ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos, expected) || ParserKeyword("CREATE TABLE").ignore(pos, expected)) + { + ParserKeyword("IF NOT EXISTS").ignore(pos, expected); + is_ddl = true; + } + else if (ParserKeyword("ALTER TABLE").ignore(pos, expected) || ParserKeyword("RENAME TABLE").ignore(pos, expected)) + { + is_ddl = true; + } + else if (ParserKeyword("DROP TABLE").ignore(pos, expected) || ParserKeyword("DROP TEMPORARY TABLE").ignore(pos, expected)) + { + ParserKeyword("IF EXISTS").ignore(pos, expected); + is_ddl = true; + } + else if (ParserKeyword("TRUNCATE").ignore(pos, expected)) + { + ParserKeyword("TABLE").ignore(pos, expected); + is_ddl = true; + } + + ASTPtr table; + if (!is_ddl || !ParserCompoundIdentifier(true).parse(pos, table, expected)) + return StorageID::createEmpty(); + auto table_id = table->as()->getTableId(); + if (table_id.database_name.empty()) + table_id.database_name = default_database_name; + return table_id; +} + +} diff --git a/src/Databases/MySQL/tryParseTableIDFromDDL.h b/src/Databases/MySQL/tryParseTableIDFromDDL.h new file mode 100644 index 00000000000..5af733f5e99 --- /dev/null +++ b/src/Databases/MySQL/tryParseTableIDFromDDL.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +StorageID tryParseTableIDFromDDL(const String & query, const String & default_database_name); + +} From 68751b88f3f3b15f5469949000ce9387cff3abdd Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 17 Jul 2023 15:01:48 +0200 Subject: [PATCH 285/473] change comments --- src/Processors/Chunk.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 7969cc7cceb..f50e45db644 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -114,7 +114,9 @@ private: using Chunks = std::vector; -/// ChunkOffsets marks offsets of different sub-chunks, which will be used by async inserts. +/// AsyncInsert needs two kinds of information: +/// - offsets of different sub-chunks +/// - tokens of different sub-chunks, which are assigned by setting `insert_deduplication_token`. class AsyncInsertInfo : public ChunkInfo { public: From 9c666552883bff611f6c15a2e7bcf3af3e0127a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:01:58 +0200 Subject: [PATCH 286/473] Fix tests --- .../test.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index 1eb2efc73a5..57ee920d49c 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -380,9 +380,9 @@ def test_table_schema_changes(started_cluster): instance.query(f"SELECT count() FROM test_database.{altered_table}") ) - cursor.execute(f"ALTER TABLE {altered_table} DROP COLUMN value2") + pg_manager.execute(f"ALTER TABLE {altered_table} DROP COLUMN value2") for i in range(NUM_TABLES): - cursor.execute(f"INSERT INTO postgresql_replica_{i} VALUES (50, {i}, {i})") + pg_manager.execute(f"INSERT INTO postgresql_replica_{i} VALUES (50, {i}, {i})") assert instance.wait_for_log_line( f"Table postgresql_replica_{altered_idx} is skipped from replication stream" @@ -556,9 +556,8 @@ def test_multiple_databases(started_cluster): port=started_cluster.postgres_port, database=False, ) - cursor = conn.cursor() - pg_manager.create_postgres_db(cursor, "postgres_database_1") - pg_manager.create_postgres_db(cursor, "postgres_database_2") + pg_manager.create_postgres_db("postgres_database_1") + pg_manager.create_postgres_db("postgres_database_2") conn1 = get_postgres_conn( ip=started_cluster.postgres_ip, @@ -577,15 +576,13 @@ def test_multiple_databases(started_cluster): cursor2 = conn2.cursor() pg_manager.create_clickhouse_postgres_db( - cluster.postgres_ip, - cluster.postgres_port, "postgres_database_1", + "", "postgres_database_1", ) pg_manager.create_clickhouse_postgres_db( - cluster.postgres_ip, - cluster.postgres_port, "postgres_database_2", + "", "postgres_database_2", ) From 815a3857de74b92d3071128a6b5fbc5cb0a53c93 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 7 Jul 2023 12:49:50 +0200 Subject: [PATCH 287/473] Remove non-const function Context::getClientInfo(). --- programs/client/Client.cpp | 11 +- programs/local/LocalServer.cpp | 5 +- src/Databases/DatabaseReplicated.cpp | 4 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 10 +- .../ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/Context.cpp | 123 +++++++++++++++ src/Interpreters/Context.h | 26 +++- src/Interpreters/DDLTask.cpp | 6 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/Session.cpp | 141 +++++++++++++++--- src/Interpreters/Session.h | 17 ++- src/Interpreters/executeQuery.cpp | 5 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 12 +- src/Server/GRPCServer.cpp | 2 +- src/Server/HTTPHandler.cpp | 19 +-- src/Server/MySQLHandler.cpp | 2 +- src/Server/PostgreSQLHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 25 +--- src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 25 files changed, 335 insertions(+), 97 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f791c39bad1..e1a33231592 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1173,12 +1173,12 @@ void Client::processOptions(const OptionsDescription & options_description, { String traceparent = options["opentelemetry-traceparent"].as(); String error; - if (!global_context->getClientInfo().client_trace_context.parseTraceparentHeader(traceparent, error)) + if (!global_context->getClientTraceContext().parseTraceparentHeader(traceparent, error)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse OpenTelemetry traceparent '{}': {}", traceparent, error); } if (options.count("opentelemetry-tracestate")) - global_context->getClientInfo().client_trace_context.tracestate = options["opentelemetry-tracestate"].as(); + global_context->getClientTraceContext().tracestate = options["opentelemetry-tracestate"].as(); } @@ -1238,10 +1238,9 @@ void Client::processConfig() global_context->getSettingsRef().max_insert_block_size); } - ClientInfo & client_info = global_context->getClientInfo(); - client_info.setInitialQuery(); - client_info.quota_key = config().getString("quota_key", ""); - client_info.query_kind = query_kind; + global_context->setQueryKindInitial(); + global_context->setQuotaClientKey(config().getString("quota_key", "")); + global_context->setQueryKind(query_kind); } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 96924e3c8d9..3c2a8ae3152 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -737,9 +737,8 @@ void LocalServer::processConfig() for (const auto & [key, value] : prompt_substitutions) boost::replace_all(prompt_by_server_display_name, "{" + key + "}", value); - ClientInfo & client_info = global_context->getClientInfo(); - client_info.setInitialQuery(); - client_info.query_kind = query_kind; + global_context->setQueryKindInitial(); + global_context->setQueryKind(query_kind); } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 661afc6bf1f..25c23e2be17 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -814,8 +814,8 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep { auto query_context = Context::createCopy(getContext()); query_context->makeQueryContext(); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context->getClientInfo().is_replicated_database_internal = true; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); + query_context->setQueryKindReplicatedDatabaseInternal(); query_context->setCurrentDatabase(getDatabaseName()); query_context->setCurrentQueryId(""); auto txn = std::make_shared(current_zookeeper, zookeeper_path, false, ""); diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index a01ab2a15a8..379e6ef5097 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -59,7 +59,7 @@ static ContextMutablePtr createQueryContext(ContextPtr context) query_context->setSettings(new_query_settings); query_context->setInternalQuery(true); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); query_context->setCurrentQueryId(""); // generate random query_id return query_context; } diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 0da762699d2..6081919a120 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -421,12 +421,10 @@ try auto insert_query_id = insert_context->getCurrentQueryId(); auto query_start_time = std::chrono::system_clock::now(); Stopwatch start_watch{CLOCK_MONOTONIC}; - ClientInfo & client_info = insert_context->getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.initial_query_start_time = timeInSeconds(query_start_time); - client_info.initial_query_start_time_microseconds = timeInMicroseconds(query_start_time); - client_info.current_query_id = insert_query_id; - client_info.initial_query_id = insert_query_id; + insert_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); + insert_context->setInitialQueryStartTime(query_start_time); + insert_context->setCurrentQueryId(insert_query_id); + insert_context->setInitialQueryId(insert_query_id); size_t log_queries_cut_to_length = insert_context->getSettingsRef().log_queries_cut_to_length; String query_for_logging = insert_query.hasSecretParts() ? insert_query.formatForLogging(log_queries_cut_to_length) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e2f1dfe8ba7..3dea52faf46 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -171,7 +171,7 @@ void executeQuery( SelectStreamFactory::Shards remote_shards; auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log); - new_context->getClientInfo().distributed_depth += 1; + new_context->increaseDistributedDepth(); size_t shards = query_info.getCluster()->getShardCount(); for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5fae9374705..c097eeb87f1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3850,6 +3850,129 @@ void Context::resetInputCallbacks() } +void Context::setClientInfo(const ClientInfo & client_info_) +{ + client_info = client_info_; + need_recalculate_access = true; +} + +void Context::setClientName(const String & client_name) +{ + client_info.client_name = client_name; +} + +void Context::setClientInterface(ClientInfo::Interface interface) +{ + client_info.interface = interface; + need_recalculate_access = true; +} + +void Context::setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + client_info.client_version_major = client_version_major; + client_info.client_version_minor = client_version_minor; + client_info.client_version_patch = client_version_patch; + client_info.client_tcp_protocol_version = client_tcp_protocol_version; +} + +void Context::setClientConnectionId(uint32_t connection_id_) +{ + client_info.connection_id = connection_id_; +} + +void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +{ + client_info.http_method = http_method; + client_info.http_user_agent = http_user_agent; + client_info.http_referer = http_referer; + need_recalculate_access = true; +} + +void Context::setForwardedFor(const String & forwarded_for) +{ + client_info.forwarded_for = forwarded_for; + need_recalculate_access = true; +} + +void Context::setQueryKind(ClientInfo::QueryKind query_kind) +{ + client_info.query_kind = query_kind; +} + +void Context::setQueryKindInitial() +{ + /// TODO: Try to combine this function with setQueryKind(). + client_info.setInitialQuery(); +} + +void Context::setQueryKindReplicatedDatabaseInternal() +{ + /// TODO: Try to combine this function with setQueryKind(). + client_info.is_replicated_database_internal = true; +} + +void Context::setCurrentUserName(const String & current_user_name) +{ + /// TODO: Try to combine this function with setUser(). + client_info.current_user = current_user_name; + need_recalculate_access = true; +} + +void Context::setCurrentAddress(const Poco::Net::SocketAddress & current_address) +{ + client_info.current_address = current_address; + need_recalculate_access = true; +} + +void Context::setInitialUserName(const String & initial_user_name) +{ + client_info.initial_user = initial_user_name; + need_recalculate_access = true; +} + +void Context::setInitialAddress(const Poco::Net::SocketAddress & initial_address) +{ + client_info.initial_address = initial_address; +} + +void Context::setInitialQueryId(const String & initial_query_id) +{ + client_info.initial_query_id = initial_query_id; +} + +void Context::setInitialQueryStartTime(std::chrono::time_point initial_query_start_time) +{ + client_info.initial_query_start_time = timeInSeconds(initial_query_start_time); + client_info.initial_query_start_time_microseconds = timeInMicroseconds(initial_query_start_time); +} + +void Context::setQuotaClientKey(const String & quota_key_) +{ + client_info.quota_key = quota_key_; + need_recalculate_access = true; +} + +void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + client_info.connection_client_version_major = client_version_major; + client_info.connection_client_version_minor = client_version_minor; + client_info.connection_client_version_patch = client_version_patch; + client_info.connection_tcp_protocol_version = client_tcp_protocol_version; +} + +void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica) +{ + client_info.collaborate_with_initiator = collaborate_with_initiator; + client_info.count_participating_replicas = all_replicas_count; + client_info.number_of_current_replica = number_of_current_replica; +} + +void Context::increaseDistributedDepth() +{ + ++client_info.distributed_depth; +} + + StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const { if (storage_id.uuid != UUIDHelpers::Nil) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 172f3818dfd..afc4bfde6a8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -593,9 +593,33 @@ public: InputBlocksReader getInputBlocksReaderCallback() const; void resetInputCallbacks(); - ClientInfo & getClientInfo() { return client_info; } + /// Returns information about the client executing a query. const ClientInfo & getClientInfo() const { return client_info; } + /// Modify stored in the context information about the client executing a query. + void setClientInfo(const ClientInfo & client_info_); + void setClientName(const String & client_name); + void setClientInterface(ClientInfo::Interface interface); + void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + void setClientConnectionId(uint32_t connection_id); + void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setForwardedFor(const String & forwarded_for); + void setQueryKind(ClientInfo::QueryKind query_kind); + void setQueryKindInitial(); + void setQueryKindReplicatedDatabaseInternal(); + void setCurrentUserName(const String & current_user_name); + void setCurrentAddress(const Poco::Net::SocketAddress & current_address); + void setInitialUserName(const String & initial_user_name); + void setInitialAddress(const Poco::Net::SocketAddress & initial_address); + void setInitialQueryId(const String & initial_query_id); + void setInitialQueryStartTime(std::chrono::time_point initial_query_start_time); + void setQuotaClientKey(const String & quota_key); + void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica); + void increaseDistributedDepth(); + const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; } + OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; } + enum StorageNamespace { ResolveGlobal = 1u, /// Database name must be specified diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index b24856a6146..4e684f5899f 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -199,7 +199,7 @@ ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const Z auto query_context = Context::createCopy(from_context); query_context->makeQueryContext(); query_context->setCurrentQueryId(""); // generate random query_id - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); if (entry.settings) query_context->applySettingsChanges(*entry.settings); return query_context; @@ -439,8 +439,8 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context) ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) { auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context->getClientInfo().is_replicated_database_internal = true; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); + query_context->setQueryKindReplicatedDatabaseInternal(); query_context->setCurrentDatabase(database->getDatabaseName()); auto txn = std::make_shared(zookeeper, database->zookeeper_path, is_initial_query, entry_path); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 81c78000ac3..193bb5b6ab0 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -476,7 +476,7 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep query_context->setSetting("implicit_transaction", Field{0}); } - query_context->getClientInfo().initial_query_id = task.entry.initial_query_id; + query_context->setInitialQueryId(task.entry.initial_query_id); if (!task.is_initial_query) query_scope.emplace(query_context); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 0beb4492aef..616cf80a446 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -451,11 +451,11 @@ void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr auto drop_context = Context::createCopy(global_context); if (ignore_sync_setting) drop_context->setSetting("database_atomic_wait_for_drop_and_detach_synchronously", false); - drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + drop_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); if (auto txn = current_context->getZooKeeperMetadataTransaction()) { /// For Replicated database - drop_context->getClientInfo().is_replicated_database_internal = true; + drop_context->setQueryKindReplicatedDatabaseInternal(); drop_context->setQueryContext(std::const_pointer_cast(current_context)); drop_context->initZooKeeperMetadataTransaction(txn, true); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 32812151b59..d07a6521544 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3183,7 +3183,7 @@ void InterpreterSelectQuery::initSettings() if (query.settings()) InterpreterSetQuery(query.settings(), context).executeForCurrentContext(options.ignore_setting_constraints); - auto & client_info = context->getClientInfo(); + const auto & client_info = context->getClientInfo(); auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; auto min_minor = DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 8571f20b91e..97b056cfc32 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -299,7 +299,10 @@ Session::~Session() if (notified_session_log_about_login) { if (auto session_log = getSessionLog()) + { + /// TODO: We have to ensure that the same info is added to the session log on a LoginSuccess event and on the corresponding Logout event. session_log->addLogOut(auth_id, user, getClientInfo()); + } } } @@ -368,17 +371,117 @@ void Session::onAuthenticationFailure(const std::optional & user_name, c } } -ClientInfo & Session::getClientInfo() -{ - /// FIXME it may produce different info for LoginSuccess and the corresponding Logout entries in the session log - return session_context ? session_context->getClientInfo() : *prepared_client_info; -} - const ClientInfo & Session::getClientInfo() const { return session_context ? session_context->getClientInfo() : *prepared_client_info; } +void Session::setClientInfo(const ClientInfo & client_info) +{ + if (session_context) + session_context->setClientInfo(client_info); + else + prepared_client_info = client_info; +} + +void Session::setClientName(const String & client_name) +{ + if (session_context) + session_context->setClientName(client_name); + else + prepared_client_info->client_name = client_name; +} + +void Session::setClientInterface(ClientInfo::Interface interface) +{ + if (session_context) + session_context->setClientInterface(interface); + else + prepared_client_info->interface = interface; +} + +void Session::setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + if (session_context) + { + session_context->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + } + else + { + prepared_client_info->client_version_major = client_version_major; + prepared_client_info->client_version_minor = client_version_minor; + prepared_client_info->client_version_patch = client_version_patch; + prepared_client_info->client_tcp_protocol_version = client_tcp_protocol_version; + } +} + +void Session::setClientConnectionId(uint32_t connection_id) +{ + if (session_context) + session_context->setClientConnectionId(connection_id); + else + prepared_client_info->connection_id = connection_id; +} + +void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +{ + if (session_context) + { + session_context->setHttpClientInfo(http_method, http_user_agent, http_referer); + } + else + { + prepared_client_info->http_method = http_method; + prepared_client_info->http_user_agent = http_user_agent; + prepared_client_info->http_referer = http_referer; + } +} + +void Session::setForwardedFor(const String & forwarded_for) +{ + if (session_context) + session_context->setForwardedFor(forwarded_for); + else + prepared_client_info->forwarded_for = forwarded_for; +} + +void Session::setQuotaClientKey(const String & quota_key) +{ + if (session_context) + session_context->setQuotaClientKey(quota_key); + else + prepared_client_info->quota_key = quota_key; +} + +void Session::setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + if (session_context) + { + session_context->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + } + else + { + prepared_client_info->connection_client_version_major = client_version_major; + prepared_client_info->connection_client_version_minor = client_version_minor; + prepared_client_info->connection_client_version_patch = client_version_patch; + prepared_client_info->connection_tcp_protocol_version = client_tcp_protocol_version; + } +} + +const OpenTelemetry::TracingContext & Session::getClientTraceContext() const +{ + if (session_context) + return session_context->getClientTraceContext(); + return prepared_client_info->client_trace_context; +} + +OpenTelemetry::TracingContext & Session::getClientTraceContext() +{ + if (session_context) + return session_context->getClientTraceContext(); + return prepared_client_info->client_trace_context; +} + ContextMutablePtr Session::makeSessionContext() { if (session_context) @@ -396,8 +499,7 @@ ContextMutablePtr Session::makeSessionContext() new_session_context->makeSessionContext(); /// Copy prepared client info to the new session context. - auto & res_client_info = new_session_context->getClientInfo(); - res_client_info = std::move(prepared_client_info).value(); + new_session_context->setClientInfo(*prepared_client_info); prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. @@ -436,8 +538,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: /// Copy prepared client info to the session context, no matter it's been just created or not. /// If we continue using a previously created session context found by session ID /// it's necessary to replace the client info in it anyway, because it contains actual connection information (client address, etc.) - auto & res_client_info = new_session_context->getClientInfo(); - res_client_info = std::move(prepared_client_info).value(); + new_session_context->setClientInfo(*prepared_client_info); prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. @@ -492,27 +593,26 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t } /// Copy the specified client info to the new query context. - auto & res_client_info = query_context->getClientInfo(); if (client_info_to_move) - res_client_info = std::move(*client_info_to_move); + query_context->setClientInfo(*client_info_to_move); else if (client_info_to_copy && (client_info_to_copy != &getClientInfo())) - res_client_info = *client_info_to_copy; + query_context->setClientInfo(*client_info_to_copy); /// Copy current user's name and address if it was authenticated after query_client_info was initialized. if (prepared_client_info && !prepared_client_info->current_user.empty()) { - res_client_info.current_user = prepared_client_info->current_user; - res_client_info.current_address = prepared_client_info->current_address; + query_context->setCurrentUserName(prepared_client_info->current_user); + query_context->setCurrentAddress(prepared_client_info->current_address); } /// Set parameters of initial query. - if (res_client_info.query_kind == ClientInfo::QueryKind::NO_QUERY) - res_client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + if (query_context->getClientInfo().query_kind == ClientInfo::QueryKind::NO_QUERY) + query_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); - if (res_client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + if (query_context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) { - res_client_info.initial_user = res_client_info.current_user; - res_client_info.initial_address = res_client_info.current_address; + query_context->setInitialUserName(query_context->getClientInfo().current_user); + query_context->setInitialAddress(query_context->getClientInfo().current_address); } /// Set user information for the new context: current profiles, roles, access rights. @@ -563,4 +663,3 @@ void Session::closeSession(const String & session_id) } } - diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index d7c06a60464..36f811ccd24 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -54,10 +54,23 @@ public: /// Writes a row about login failure into session log (if enabled) void onAuthenticationFailure(const std::optional & user_name, const Poco::Net::SocketAddress & address_, const Exception & e); - /// Returns a reference to session ClientInfo. - ClientInfo & getClientInfo(); + /// Returns a reference to the session's ClientInfo. const ClientInfo & getClientInfo() const; + /// Modify the session's ClientInfo. + void setClientInfo(const ClientInfo & client_info); + void setClientName(const String & client_name); + void setClientInterface(ClientInfo::Interface interface); + void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + void setClientConnectionId(uint32_t connection_id); + void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setForwardedFor(const String & forwarded_for); + void setQuotaClientKey(const String & quota_key); + void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + + const OpenTelemetry::TracingContext & getClientTraceContext() const; + OpenTelemetry::TracingContext & getClientTraceContext(); + /// Makes a session context, can be used one or zero times. /// The function also assigns an user to this context. ContextMutablePtr makeSessionContext(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b76d20f31d..2c74039463e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -655,7 +655,7 @@ static std::tuple executeQueryImpl( /// the value passed by the client Stopwatch start_watch{CLOCK_MONOTONIC}; - auto & client_info = context->getClientInfo(); + const auto & client_info = context->getClientInfo(); if (!internal) { @@ -667,8 +667,7 @@ static std::tuple executeQueryImpl( // On the other hand, if it's initialized then take it as the start of the query if (client_info.initial_query_start_time == 0) { - client_info.initial_query_start_time = timeInSeconds(query_start_time); - client_info.initial_query_start_time_microseconds = timeInMicroseconds(query_start_time); + context->setInitialQueryStartTime(query_start_time); } else { diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 9b9cc221ca8..b251eec2d28 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -72,14 +72,10 @@ std::unique_ptr createLocalPlan( if (coordinator) { new_context->parallel_reading_coordinator = coordinator; - new_context->getClientInfo().interface = ClientInfo::Interface::LOCAL; - new_context->getClientInfo().collaborate_with_initiator = true; - new_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - new_context->getClientInfo().count_participating_replicas = replica_count; - new_context->getClientInfo().number_of_current_replica = replica_num; - new_context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR; - new_context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR; - new_context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION; + new_context->setClientInterface(ClientInfo::Interface::LOCAL); + new_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); + new_context->setReplicaInfo(true, replica_count, replica_num); + new_context->setConnectionClientVersion(DBMS_VERSION_MAJOR, DBMS_VERSION_MINOR, DBMS_VERSION_PATCH, DBMS_TCP_PROTOCOL_VERSION); new_context->setParallelReplicasGroupUUID(group_uuid); new_context->setMergeTreeAllRangesCallback([coordinator](InitialAllRangesAnnouncement announcement) { diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index bf9ba20a5cf..67d30012b0e 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -798,7 +798,7 @@ namespace /// Authentication. session.emplace(iserver.context(), ClientInfo::Interface::GRPC); session->authenticate(user, password, user_address); - session->getClientInfo().quota_key = quota_key; + session->setQuotaClientKey(quota_key); ClientInfo client_info = session->getClientInfo(); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index f7cdb905710..069670c84a5 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -474,7 +474,6 @@ bool HTTPHandler::authenticateUser( } /// Set client info. It will be used for quota accounting parameters in 'setUser' method. - ClientInfo & client_info = session->getClientInfo(); ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; if (request.getMethod() == HTTPServerRequest::HTTP_GET) @@ -482,15 +481,13 @@ bool HTTPHandler::authenticateUser( else if (request.getMethod() == HTTPServerRequest::HTTP_POST) http_method = ClientInfo::HTTPMethod::POST; - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); - client_info.http_referer = request.get("Referer", ""); - client_info.forwarded_for = request.get("X-Forwarded-For", ""); - client_info.quota_key = quota_key; + session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); + session->setForwardedFor(request.get("X-Forwarded-For", "")); + session->setQuotaClientKey(quota_key); /// Extract the last entry from comma separated list of forwarded_for addresses. /// Only the last proxy can be trusted (if any). - String forwarded_address = client_info.getLastForwardedFor(); + String forwarded_address = session->getClientInfo().getLastForwardedFor(); try { if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) @@ -988,22 +985,22 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse } // Parse the OpenTelemetry traceparent header. - ClientInfo& client_info = session->getClientInfo(); + auto & client_trace_context = session->getClientTraceContext(); if (request.has("traceparent")) { std::string opentelemetry_traceparent = request.get("traceparent"); std::string error; - if (!client_info.client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) + if (!client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) { LOG_DEBUG(log, "Failed to parse OpenTelemetry traceparent header '{}': {}", opentelemetry_traceparent, error); } - client_info.client_trace_context.tracestate = request.get("tracestate", ""); + client_trace_context.tracestate = request.get("tracestate", ""); } // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); thread_trace_context = std::make_unique("HTTPHandler", - client_info.client_trace_context, + client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog()); thread_trace_context->root_span.kind = OpenTelemetry::SERVER; diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 7318b0ad89b..f98b86e6cf8 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -94,7 +94,7 @@ void MySQLHandler::run() session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); - session->getClientInfo().connection_id = connection_id; + session->setClientConnectionId(connection_id); in = std::make_shared(socket()); out = std::make_shared(socket()); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 36b05932979..7b078154252 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -58,7 +58,7 @@ void PostgreSQLHandler::run() session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); - session->getClientInfo().connection_id = connection_id; + session->setClientConnectionId(connection_id); try { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 36566832ebc..a747f06f1ce 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1177,21 +1177,12 @@ std::unique_ptr TCPHandler::makeSession() auto res = std::make_unique(server.context(), interface, socket().secure(), certificate); - auto & client_info = res->getClientInfo(); - client_info.forwarded_for = forwarded_for; - client_info.client_name = client_name; - client_info.client_version_major = client_version_major; - client_info.client_version_minor = client_version_minor; - client_info.client_version_patch = client_version_patch; - client_info.client_tcp_protocol_version = client_tcp_protocol_version; - - client_info.connection_client_version_major = client_version_major; - client_info.connection_client_version_minor = client_version_minor; - client_info.connection_client_version_patch = client_version_patch; - client_info.connection_tcp_protocol_version = client_tcp_protocol_version; - - client_info.quota_key = quota_key; - client_info.interface = interface; + res->setForwardedFor(forwarded_for); + res->setClientName(client_name); + res->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + res->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + res->setQuotaClientKey(quota_key); + res->setClientInterface(interface); return res; } @@ -1253,7 +1244,7 @@ void TCPHandler::receiveHello() } session = makeSession(); - auto & client_info = session->getClientInfo(); + const auto & client_info = session->getClientInfo(); #if USE_SSL /// Authentication with SSL user certificate @@ -1286,7 +1277,7 @@ void TCPHandler::receiveAddendum() { readStringBinary(quota_key, *in); if (!is_interserver_mode) - session->getClientInfo().quota_key = quota_key; + session->setQuotaClientKey(quota_key); } } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 875764f7633..0dcdae01ba9 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -132,7 +132,7 @@ DistributedSink::DistributedSink( const auto & settings = context->getSettingsRef(); if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception(ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH, "Maximum distributed depth exceeded"); - context->getClientInfo().distributed_depth += 1; + context->increaseDistributedDepth(); random_shard_insert = settings.insert_distributed_one_random_shard && !storage.has_sharding_key; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b..c028cf5ec77 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -914,7 +914,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu QueryPipeline pipeline; ContextMutablePtr query_context = Context::createCopy(local_context); - ++query_context->getClientInfo().distributed_depth; + query_context->increaseDistributedDepth(); for (size_t shard_index : collections::range(0, shards_info.size())) { @@ -976,7 +976,7 @@ std::optional StorageDistributed::distributedWriteFromClusterStor QueryPipeline pipeline; ContextMutablePtr query_context = Context::createCopy(local_context); - ++query_context->getClientInfo().distributed_depth; + query_context->increaseDistributedDepth(); /// Here we take addresses from destination cluster and assume source table exists on these nodes for (const auto & replicas : getCluster()->getShardsAddresses()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..066f5a42f46 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5079,7 +5079,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu QueryPipeline pipeline; ContextMutablePtr query_context = Context::createCopy(local_context); - ++query_context->getClientInfo().distributed_depth; + query_context->increaseDistributedDepth(); for (const auto & replicas : src_cluster->getShardsAddresses()) { diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 242e8e5d570..0f506040cd9 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -992,7 +992,7 @@ void StorageWindowView::cleanup() auto cleanup_context = Context::createCopy(getContext()); cleanup_context->makeQueryContext(); cleanup_context->setCurrentQueryId(""); - cleanup_context->getClientInfo().is_replicated_database_internal = true; + cleanup_context->setQueryKindReplicatedDatabaseInternal(); InterpreterAlterQuery interpreter_alter(alter_query, cleanup_context); interpreter_alter.execute(); From 7a224c252ef9e544638354f0afa15d6276c61e59 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:04:39 +0200 Subject: [PATCH 288/473] One more fix --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 0514c461c5e..97eb5ca7122 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -695,6 +695,7 @@ def test_too_many_parts(started_cluster): break time.sleep(1) print(f"wait sync try {i}") + instance2.query("SYSTEM FLUSH LOGS") if instance2.contains_in_log("DB::Exception: Too many parts"): num = num - 1 break From 2379d8c9d5f73be50c78978c73aea1c2c87044e0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jul 2023 14:52:17 +0200 Subject: [PATCH 289/473] Revert unnecessary improving in ContextAccessCache for now. --- src/Access/AccessControl.cpp | 24 +++++++++--------------- 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 41ac3f42ee2..bf0a2a0fbba 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -72,26 +72,20 @@ public: std::shared_ptr getContextAccess(const ContextAccessParams & params) { + std::lock_guard lock{mutex}; + auto x = cache.get(params); + if (x) { - std::lock_guard lock{mutex}; - auto x = cache.get(params); - if (x) - { - if ((*x)->getUserID() && !(*x)->tryGetUser()) - cache.remove(params); /// The user has been dropped while it was in the cache. - else - return *x; - } + if ((*x)->getUserID() && !(*x)->tryGetUser()) + cache.remove(params); /// The user has been dropped while it was in the cache. + else + return *x; } + /// TODO: There is no need to keep the `ContextAccessCache::mutex` locked while we're calculating access rights. auto res = std::make_shared(access_control, params); res->initialize(); - - { - std::lock_guard lock{mutex}; - cache.add(params, res); - } - + cache.add(params, res); return res; } From d36d69c23cc8d123ac98a8710eb4712a21a30d87 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Mon, 17 Jul 2023 14:27:41 +0200 Subject: [PATCH 290/473] MaterializedMySQL: Replace to_string by magic_enum::enum_name --- src/Core/MySQL/MySQLReplication.cpp | 4 +- src/Core/MySQL/MySQLReplication.h | 112 ---------------------------- 2 files changed, 2 insertions(+), 114 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1ee027b7185..70a8af78c04 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -40,9 +40,9 @@ namespace MySQLReplication void EventHeader::dump(WriteBuffer & out) const { - out << "\n=== " << to_string(this->type) << " ===" << '\n'; + out << "\n=== " << magic_enum::enum_name(this->type) << " ===" << '\n'; out << "Timestamp: " << this->timestamp << '\n'; - out << "Event Type: " << to_string(this->type) << '\n'; + out << "Event Type: " << magic_enum::enum_name(this->type) << '\n'; out << "Server ID: " << this->server_id << '\n'; out << "Event Size: " << this->event_size << '\n'; out << "Log Pos: " << this->log_pos << '\n'; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 5825924d10b..7e19b0ea11b 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -120,22 +120,6 @@ namespace MySQLReplication BINLOG_CHECKSUM_ALG_UNDEF = 255 }; - inline String to_string(BinlogChecksumAlg type) - { - switch (type) - { - case BINLOG_CHECKSUM_ALG_OFF: - return "BINLOG_CHECKSUM_ALG_OFF"; - case BINLOG_CHECKSUM_ALG_CRC32: - return "BINLOG_CHECKSUM_ALG_CRC32"; - case BINLOG_CHECKSUM_ALG_ENUM_END: - return "BINLOG_CHECKSUM_ALG_ENUM_END"; - case BINLOG_CHECKSUM_ALG_UNDEF: - return "BINLOG_CHECKSUM_ALG_UNDEF"; - } - return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); - } - /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html enum EventType { @@ -187,102 +171,6 @@ namespace MySQLReplication MARIA_START_ENCRYPTION_EVENT = 164, }; - inline String to_string(EventType type) - { - switch (type) - { - case START_EVENT_V3: - return "StartEventV3"; - case QUERY_EVENT: - return "QueryEvent"; - case STOP_EVENT: - return "StopEvent"; - case ROTATE_EVENT: - return "RotateEvent"; - case INT_VAR_EVENT: - return "IntVarEvent"; - case LOAD_EVENT: - return "LoadEvent"; - case SLAVE_EVENT: - return "SlaveEvent"; - case CREATE_FILE_EVENT: - return "CreateFileEvent"; - case APPEND_BLOCK_EVENT: - return "AppendBlockEvent"; - case EXEC_LOAD_EVENT: - return "ExecLoadEvent"; - case DELETE_FILE_EVENT: - return "DeleteFileEvent"; - case NEW_LOAD_EVENT: - return "NewLoadEvent"; - case RAND_EVENT: - return "RandEvent"; - case USER_VAR_EVENT: - return "UserVarEvent"; - case FORMAT_DESCRIPTION_EVENT: - return "FormatDescriptionEvent"; - case XID_EVENT: - return "XIDEvent"; - case BEGIN_LOAD_QUERY_EVENT: - return "BeginLoadQueryEvent"; - case EXECUTE_LOAD_QUERY_EVENT: - return "ExecuteLoadQueryEvent"; - case TABLE_MAP_EVENT: - return "TableMapEvent"; - case WRITE_ROWS_EVENT_V0: - return "WriteRowsEventV0"; - case UPDATE_ROWS_EVENT_V0: - return "UpdateRowsEventV0"; - case DELETE_ROWS_EVENT_V0: - return "DeleteRowsEventV0"; - case WRITE_ROWS_EVENT_V1: - return "WriteRowsEventV1"; - case UPDATE_ROWS_EVENT_V1: - return "UpdateRowsEventV1"; - case DELETE_ROWS_EVENT_V1: - return "DeleteRowsEventV1"; - case INCIDENT_EVENT: - return "IncidentEvent"; - case HEARTBEAT_EVENT: - return "HeartbeatEvent"; - case IGNORABLE_EVENT: - return "IgnorableEvent"; - case ROWS_QUERY_EVENT: - return "RowsQueryEvent"; - case WRITE_ROWS_EVENT_V2: - return "WriteRowsEventV2"; - case UPDATE_ROWS_EVENT_V2: - return "UpdateRowsEventV2"; - case DELETE_ROWS_EVENT_V2: - return "DeleteRowsEventV2"; - case GTID_EVENT: - return "GTIDEvent"; - case ANONYMOUS_GTID_EVENT: - return "AnonymousGTIDEvent"; - case PREVIOUS_GTIDS_EVENT: - return "PreviousGTIDsEvent"; - case TRANSACTION_CONTEXT_EVENT: - return "TransactionContextEvent"; - case VIEW_CHANGE_EVENT: - return "ViewChangeEvent"; - case XA_PREPARE_LOG_EVENT: - return "XAPrepareLogEvent"; - case MARIA_ANNOTATE_ROWS_EVENT: - return "MariaAnnotateRowsEvent"; - case MARIA_BINLOG_CHECKPOINT_EVENT: - return "MariaBinlogCheckpointEvent"; - case MARIA_GTID_EVENT: - return "MariaGTIDEvent"; - case MARIA_GTID_LIST_EVENT: - return "MariaGTIDListEvent"; - case MARIA_START_ENCRYPTION_EVENT: - return "MariaStartEncryptionEvent"; - default: - break; - } - return std::string("Unknown event: ") + std::to_string(static_cast(type)); - } - enum MySQLEventType { MYSQL_UNHANDLED_EVENT = 0, From dd74ce3e1cd62444dc8b7c2b09ac7c5bc58f68eb Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 17 Jul 2023 13:44:52 +0000 Subject: [PATCH 291/473] Add integration tests check with analyzer to master --- .github/workflows/master.yml | 210 +++++++++++++++++++++++++++++++++++ 1 file changed, 210 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 0fbcb95fc12..4254d899c6f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2870,6 +2870,216 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan3: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan4: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan5: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=5 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" IntegrationTestsTsan0: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] From 8f8cb5eb8289af1f720b976158da364227619250 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 15:48:12 +0200 Subject: [PATCH 292/473] Update cmake/limit_jobs.cmake MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- cmake/limit_jobs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 4d81619aa13..39a6f688488 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -43,7 +43,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") +message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs.") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From fc05187e8a61f199c7eb5134edc6f7bbf9a9d981 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 17 Jul 2023 13:50:21 +0000 Subject: [PATCH 293/473] Add dependencies to FinishCheck --- .github/workflows/master.yml | 6 ++++++ .github/workflows/pull_request.yml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4254d899c6f..e5b797beebd 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -4173,6 +4173,12 @@ jobs: - IntegrationTestsAsan3 - IntegrationTestsAsan4 - IntegrationTestsAsan5 + - IntegrationTestsAnalyzerAsan0 + - IntegrationTestsAnalyzerAsan1 + - IntegrationTestsAnalyzerAsan2 + - IntegrationTestsAnalyzerAsan3 + - IntegrationTestsAnalyzerAsan4 + - IntegrationTestsAnalyzerAsan5 - IntegrationTestsRelease0 - IntegrationTestsRelease1 - IntegrationTestsRelease2 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index d8030c12128..dd834959578 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5099,6 +5099,12 @@ jobs: - IntegrationTestsAsan3 - IntegrationTestsAsan4 - IntegrationTestsAsan5 + - IntegrationTestsAnalyzerAsan0 + - IntegrationTestsAnalyzerAsan1 + - IntegrationTestsAnalyzerAsan2 + - IntegrationTestsAnalyzerAsan3 + - IntegrationTestsAnalyzerAsan4 + - IntegrationTestsAnalyzerAsan5 - IntegrationTestsRelease0 - IntegrationTestsRelease1 - IntegrationTestsRelease2 From 2ad1109dbdf37f9a854ea2dbe954312101c127b7 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 16:03:39 +0200 Subject: [PATCH 294/473] Avoid zero copy replication more agressively --- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..40d01215498 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2448,10 +2448,13 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (part_desc->checksum_hex != part_desc->src_table_part->checksums.getTotalChecksumHex()) throw Exception(ErrorCodes::UNFINISHED, "Checksums of {} is suddenly changed", part_desc->src_table_part->name); - bool zero_copy_enabled = dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + /// Don't do hardlinks in case of zero-copy at any side (defensive programming) + bool source_zero_copy_enabled = dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + bool our_zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(), + .copy_instead_of_hardlink = (our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( @@ -7585,8 +7588,10 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + /// Don't do hardlinks in case of zero-copy at any side (defensive programming) bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(dest_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(), From 9a30aaa9aa1924d35d79806fb2090570e0d9df49 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 17 Jul 2023 16:14:58 +0200 Subject: [PATCH 295/473] Revert "Fixed several issues found by OSS-Fuzz" --- ..._function_state_deserialization_fuzzer.cpp | 1 - .../fuzzers/delta_decompress_fuzzer.cpp | 2 +- .../double_delta_decompress_fuzzer.cpp | 2 +- .../fuzzers/encrypted_decompress_fuzzer.cpp | 4 +- .../fuzzers/lz4_decompress_fuzzer.cpp | 4 +- src/Core/Field.h | 66 +------------------ src/DataTypes/DataTypeFactory.cpp | 2 +- src/Functions/DateTimeTransforms.h | 4 +- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- 9 files changed, 10 insertions(+), 77 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 3db1afb7a92..2ea01e1d5bc 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -8,7 +8,6 @@ #include #include -#include #include diff --git a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp index 451606843e2..b039777da15 100644 --- a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp @@ -34,7 +34,7 @@ try DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); - codec->doDecompressData(reinterpret_cast(data), static_cast(size), memory.data(), static_cast(output_buffer_size)); + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); return 0; } diff --git a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp index f7e685d68ad..f9822daa3bd 100644 --- a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp @@ -34,7 +34,7 @@ try DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); - codec->doDecompressData(reinterpret_cast(data), static_cast(size), memory.data(), static_cast(output_buffer_size)); + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); return 0; } diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp index 207cce21e3b..3e3d0e164fe 100644 --- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -292,10 +292,10 @@ try DB::Memory<> memory; memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer()); - codec_128->doDecompressData(input.data(), static_cast(input.size()), memory.data(), static_cast(input.size() - 31)); + codec_128->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31); memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer()); - codec_256->doDecompressData(input.data(), static_cast(input.size()), memory.data(), static_cast(input.size() - 31)); + codec_256->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31); return 0; } catch (...) diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp index f6d4c51f18b..85c4c9bd329 100644 --- a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -24,7 +24,7 @@ try return 0; const auto * p = reinterpret_cast(data); - auto codec = DB::getCompressionCodecLZ4(static_cast(p->level)); + auto codec = DB::getCompressionCodecLZ4(p->level); size_t output_buffer_size = p->decompressed_size % 65536; size -= sizeof(AuxiliaryRandomData); @@ -37,7 +37,7 @@ try DB::Memory<> memory; memory.resize(output_buffer_size + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - codec->doDecompressData(reinterpret_cast(data), static_cast(size), memory.data(), static_cast(output_buffer_size)); + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); return 0; } diff --git a/src/Core/Field.h b/src/Core/Field.h index cc0083e02d8..ef1bd9a895d 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int TOO_DEEP_RECURSION; } constexpr Null NEGATIVE_INFINITY{Null::Value::NegativeInfinity}; @@ -42,13 +41,10 @@ using FieldVector = std::vector>; /// construct a Field of Array or a Tuple type. An alternative approach would be /// to construct both of these types from FieldVector, and have the caller /// specify the desired Field type explicitly. -/// As the result stack overflow on destruction is possible -/// and to avoid it we need to count the depth and have a threshold. #define DEFINE_FIELD_VECTOR(X) \ struct X : public FieldVector \ { \ using FieldVector::FieldVector; \ - uint8_t nested_field_depth = 0; \ } DEFINE_FIELD_VECTOR(Array); @@ -65,7 +61,6 @@ using FieldMap = std::map, AllocatorWithMemoryTrackin struct X : public FieldMap \ { \ using FieldMap::FieldMap; \ - uint8_t nested_field_depth = 0; \ } DEFINE_FIELD_MAP(Object); @@ -296,12 +291,6 @@ decltype(auto) castToNearestFieldType(T && x) */ #define DBMS_MIN_FIELD_SIZE 32 -/// Note: uint8_t is used for storing depth value. -#if defined(SANITIZER) || !defined(NDEBUG) - #define DBMS_MAX_NESTED_FIELD_DEPTH 64 -#else - #define DBMS_MAX_NESTED_FIELD_DEPTH 255 -#endif /** Discriminated union of several types. * Made for replacement of `boost::variant` @@ -682,49 +671,6 @@ private: Types::Which which; - /// StorageType and Original are the same for Array, Tuple, Map, Object - template - uint8_t calculateAndCheckFieldDepth(Original && x) - { - uint8_t result = 0; - - if constexpr (std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - { - result = x.nested_field_depth; - - auto get_depth = [](const Field & elem) - { - switch (elem.which) - { - case Types::Array: - return elem.template get().nested_field_depth; - case Types::Tuple: - return elem.template get().nested_field_depth; - case Types::Map: - return elem.template get().nested_field_depth; - case Types::Object: - return elem.template get().nested_field_depth; - default: - return static_cast(0); - } - }; - - if constexpr (std::is_same_v) - for (auto & [_, value] : x) - result = std::max(get_depth(value), result); - else - for (auto & value : x) - result = std::max(get_depth(value), result); - } - - if (result >= DBMS_MAX_NESTED_FIELD_DEPTH) - throw Exception(ErrorCodes::TOO_DEEP_RECURSION, "Too deep Field"); - - return result; - } /// Assuming there was no allocated state or it was deallocated (see destroy). template @@ -738,17 +684,7 @@ private: // we must initialize the entire wide stored type, and not just the // nominal type. using StorageType = NearestFieldType; - - /// Incrementing the depth since we create a new Field. - auto depth = calculateAndCheckFieldDepth(x); new (&storage) StorageType(std::forward(x)); - - if constexpr (std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - reinterpret_cast(&storage)->nested_field_depth = depth + 1; - which = TypeToEnum::value; } @@ -845,7 +781,7 @@ private: } template - ALWAYS_INLINE void destroy() + void destroy() { T * MAY_ALIAS ptr = reinterpret_cast(&storage); ptr->~T(); diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 89dacae59ff..415f24d8151 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -62,7 +62,7 @@ DataTypePtr DataTypeFactory::getImpl(const String & full_name) const } else { - ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", DBMS_DEFAULT_MAX_QUERY_SIZE, data_type_max_parse_depth); + ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", false, data_type_max_parse_depth); } return getImpl(ast); diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 0d71e802bf0..510a88db2b6 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1521,10 +1521,8 @@ struct Transformer if constexpr (std::is_same_v || std::is_same_v) { -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" bool is_valid_input = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; -# pragma clang diagnostic pop + if (!is_valid_input) { if constexpr (std::is_same_v) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 30f0e91a75b..727c49cfc4d 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -42,4 +42,4 @@ clickhouse_add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") target_include_directories(codegen_select_fuzzer SYSTEM BEFORE PRIVATE "${CMAKE_CURRENT_BINARY_DIR}") -target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf ch_contrib::protobuf_mutator ch_contrib::protoc dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc dbms ${LIB_FUZZING_ENGINE}) From 2a2ebb1f318ce665885ac2c5f711d6a29fbed582 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 17 Jul 2023 14:26:11 +0000 Subject: [PATCH 296/473] Bump From fff1ae73691bc3bbd409b1743a2c85d18412f868 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jul 2023 17:08:36 +0200 Subject: [PATCH 297/473] Use default destructor for ContextAccess. --- src/Access/ContextAccess.cpp | 30 +++++++++--------------------- 1 file changed, 9 insertions(+), 21 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 51bb7794735..9e9d8644539 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -235,20 +235,7 @@ ContextAccess::ContextAccess(const AccessControl & access_control_, const Params } -ContextAccess::~ContextAccess() -{ - enabled_settings.reset(); - enabled_quota.reset(); - enabled_row_policies.reset(); - row_policies_of_initial_user.reset(); - access_with_implicit.reset(); - access.reset(); - roles_info.reset(); - subscription_for_roles_changes.reset(); - enabled_roles.reset(); - subscription_for_user_change.reset(); - user.reset(); -} +ContextAccess::~ContextAccess() = default; void ContextAccess::initialize() @@ -265,12 +252,6 @@ void ContextAccess::initialize() if (!params.user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug"); - if (!params.initial_user.empty()) - { - if (auto initial_user_id = access_control->find(params.initial_user)) - row_policies_of_initial_user = access_control->tryGetDefaultRowPolicies(*initial_user_id); - } - subscription_for_user_change = access_control->subscribeForChanges( *params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) @@ -290,7 +271,8 @@ void ContextAccess::initialize() void ContextAccess::setUser(const UserPtr & user_) const { user = user_; - if (!user) + + if (!user_) { /// User has been dropped. user_was_dropped = true; @@ -301,6 +283,7 @@ void ContextAccess::setUser(const UserPtr & user_) const enabled_roles = nullptr; roles_info = nullptr; enabled_row_policies = nullptr; + row_policies_of_initial_user = nullptr; enabled_quota = nullptr; enabled_settings = nullptr; return; @@ -330,6 +313,11 @@ void ContextAccess::setUser(const UserPtr & user_) const }); setRolesInfo(enabled_roles->getRolesInfo()); + + std::optional initial_user_id; + if (!params.initial_user.empty()) + initial_user_id = access_control->find(params.initial_user); + row_policies_of_initial_user = initial_user_id ? access_control->tryGetDefaultRowPolicies(*initial_user_id) : nullptr; } From 2abd69d9216a62e2e4ac2ff36e45108d89b741db Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 17 Jul 2023 15:14:54 +0000 Subject: [PATCH 298/473] Fix data race on Connection::setAsyncCallback --- src/Client/MultiplexedConnections.cpp | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index e5c4a19e02a..1a26c4609c7 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -319,24 +319,21 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "Logical error: no available replica"); Packet packet; + try { AsyncCallbackSetter async_setter(current_connection, std::move(async_callback)); - - try + packet = current_connection->receivePacket(); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_SERVER) { - packet = current_connection->receivePacket(); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_SERVER) - { - /// Exception may happen when packet is received, e.g. when got unknown packet. - /// In this case, invalidate replica, so that we would not read from it anymore. - current_connection->disconnect(); - invalidateReplica(state); - } - throw; + /// Exception may happen when packet is received, e.g. when got unknown packet. + /// In this case, invalidate replica, so that we would not read from it anymore. + current_connection->disconnect(); + invalidateReplica(state); } + throw; } switch (packet.type) From 1dbb86a045a3e505cf7e71de45e992cc1b288bd3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 13:38:46 +0200 Subject: [PATCH 299/473] Add _gen to ignore (generated by pytest) Signed-off-by: Azat Khuzhin --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index a04c60d5ca3..39d6f3f9fc8 100644 --- a/.gitignore +++ b/.gitignore @@ -161,6 +161,7 @@ tests/queries/0_stateless/test_* tests/queries/0_stateless/*.binary tests/queries/0_stateless/*.generated-expect tests/queries/0_stateless/*.expect.history +tests/integration/**/_gen # rust /rust/**/target From 6f6562c96bf5fb7157ff582815bae1330d34c50f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:23:40 +0200 Subject: [PATCH 300/473] Fix running integration tests with spaces in it's names Previously they were simply ignored, and not only them, but all the tests in that group, here is an example [1] [1]: https://s3.amazonaws.com/clickhouse-test-reports/51448/4ed462ac7834a8180f92ca7d7d3c076e687bfca9/integration_tests__asan__[4_6].html Note, that I've replaced PYTEST_OPTS with handled by pytest itself PYTEST_ADDOPTS, since "sh -c $PYTEST_OPTS" does not allows you to preserve the arguments correctly. v5: use double quotes for PYTEST_ADDOPTS value Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/Dockerfile | 3 ++- tests/integration/ci-runner.py | 3 ++- tests/integration/runner | 13 ++++++++++--- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 0d1fa00b214..8e95d94b6dc 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -135,4 +135,5 @@ ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] -CMD ["sh", "-c", "pytest $PYTEST_OPTS"] +# To pass additional arguments (i.e. list of tests) use PYTEST_ADDOPTS +CMD ["sh", "-c", "pytest"] diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 31f4a7666c8..98675cb0b34 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -10,6 +10,7 @@ import random import shutil import subprocess import time +import shlex import zlib # for crc32 @@ -646,7 +647,7 @@ class ClickhouseIntegrationTestsRunner: info_basename = test_group_str + "_" + str(i) + ".nfo" info_path = os.path.join(repo_path, "tests/integration", info_basename) - test_cmd = " ".join([test for test in sorted(test_names)]) + test_cmd = " ".join([shlex.quote(test) for test in sorted(test_names)]) parallel_cmd = ( " --parallel {} ".format(num_workers) if num_workers > 0 else "" ) diff --git a/tests/integration/runner b/tests/integration/runner index c124ad46447..d524948dbcf 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -11,6 +11,7 @@ import subprocess import sys import string import random +import shlex def random_str(length=6): @@ -407,8 +408,14 @@ if __name__ == "__main__": if args.analyzer: use_analyzer = "-e CLICKHOUSE_USE_NEW_ANALYZER=1" - pytest_opts = " ".join(args.pytest_args).replace("'", "\\'") - tests_list = " ".join(args.tests_list) + # NOTE: since pytest options is in the argument value already we need to additionally escape '"' + pytest_opts = " ".join( + map(lambda x: shlex.quote(x).replace('"', '\\"'), args.pytest_args) + ) + tests_list = " ".join( + map(lambda x: shlex.quote(x).replace('"', '\\"'), args.tests_list) + ) + cmd_base = ( f"docker run {net} {tty} --rm --name {CONTAINER_NAME} " "--privileged --dns-search='.' " # since recent dns search leaks from host @@ -420,7 +427,7 @@ if __name__ == "__main__": f"--volume={args.src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos " f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} " f"-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 {use_analyzer} -e PYTHONUNBUFFERED=1 " - f"-e PYTEST_OPTS='{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv'" + f'-e PYTEST_ADDOPTS="{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv"' f" {DIND_INTEGRATION_TESTS_IMAGE_NAME}:{args.docker_image_version}" ) From e532d5b376ebadaa8fad5dd5c026e082bf851b7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 13:48:15 +0200 Subject: [PATCH 301/473] tests: use -- over quotes on and on in ci-runner.py Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 98675cb0b34..4f40058b3d9 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -412,7 +412,7 @@ class ClickhouseIntegrationTestsRunner: out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log") cmd = ( "cd {repo_path}/tests/integration && " - "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " + "timeout -s 9 1h ./runner {runner_opts} {image_cmd} -- --setup-plan " "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " "| grep -v 'SKIPPED' | sort -u > {out_file}".format( repo_path=repo_path, @@ -656,7 +656,7 @@ class ClickhouseIntegrationTestsRunner: # -E -- (E)rror # -p -- (p)assed # -s -- (s)kipped - cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} '-rfEps --run-id={} --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} -- -rfEps --run-id={} --color=no --durations=0 {} | tee {}".format( repo_path, self._get_runner_opts(), image_cmd, From acbdbf6ab744ec4659c7ed3ac613c1487a5ab021 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 13:49:03 +0200 Subject: [PATCH 302/473] tests: do not use --format if --quiet is set in integration runner Signed-off-by: Azat Khuzhin --- tests/integration/runner | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index d524948dbcf..1b902803741 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -136,9 +136,7 @@ def check_args_and_update_paths(args): def docker_kill_handler_handler(signum, frame): subprocess.check_call( - 'docker ps --all --quiet --filter name={name} --format="{{{{.ID}}}}"'.format( - name=CONTAINER_NAME - ), + "docker ps --all --quiet --filter name={name}".format(name=CONTAINER_NAME), shell=True, ) raise KeyboardInterrupt("Killed by Ctrl+C") @@ -438,7 +436,7 @@ if __name__ == "__main__": ) containers = subprocess.check_output( - f"docker ps --all --quiet --filter name={CONTAINER_NAME} --format={{{{.ID}}}}", + f"docker ps --all --quiet --filter name={CONTAINER_NAME}", shell=True, universal_newlines=True, ).splitlines() From 6c986c1fb22e2a5fff2bc281d55eb545868a4645 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 12 Jul 2023 15:21:28 +0200 Subject: [PATCH 303/473] Allow test names with spaces in it It had been fixed and now it is allowed. Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 41 +++++++++++++++++++++------------- 1 file changed, 26 insertions(+), 15 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4f40058b3d9..f2adea78da7 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -7,6 +7,7 @@ import json import logging import os import random +import re import shutil import subprocess import time @@ -111,16 +112,36 @@ def get_counters(fname): if not (".py::" in line and " " in line): continue - line_arr = line.strip().split(" ") + line = line.strip() + # [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client + # ^^^^^^^^^^^^^ + if line.strip().startswith("["): + line = re.sub("^\[[^\[\]]*\] \[[^\[\]]*\] ", "", line) + + line_arr = line.split(" ") if len(line_arr) < 2: logging.debug("Strange line %s", line) continue # Lines like: - # [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client - # [gw3] [ 40%] PASSED test_replicated_users/test.py::test_rename_replicated[QUOTA] - state = line_arr[-2] - test_name = line_arr[-1] + # + # ERROR test_mysql_protocol/test.py::test_golang_client + # PASSED test_replicated_users/test.py::test_rename_replicated[QUOTA] + # PASSED test_drop_is_lock_free/test.py::test_query_is_lock_free[detach part] + # + state = line_arr.pop(0) + test_name = " ".join(line_arr) + + # Normalize test names for lines like this: + # + # FAILED test_storage_s3/test.py::test_url_reconnect_in_the_middle - Exception + # FAILED test_distributed_ddl/test.py::test_default_database[configs] - AssertionError: assert ... + # + test_name = re.sub( + r"^(?P[^\[\] ]+)(?P\[[^\[\]]*\]|)(?P - .*|)$", + r"\g\g", + test_name, + ) if state in counters: counters[state].add(test_name) @@ -1000,16 +1021,6 @@ class ClickhouseIntegrationTestsRunner: if "(memory)" in self.params["context_name"]: result_state = "success" - for res in test_result: - # It's not easy to parse output of pytest - # Especially when test names may contain spaces - # Do not allow it to avoid obscure failures - if " " not in res[0]: - continue - logging.warning("Found invalid test name with space: %s", res[0]) - status_text = "Found test with invalid name, see main log" - result_state = "failure" - return result_state, status_text, test_result, [] From ee5e639ce9ea4ed07f9fb309c9dbc6c1ad2193a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 17:29:47 +0200 Subject: [PATCH 304/473] Update analyzer integration broken tests list After proper parsing more failed tests pops up: - https://s3.amazonaws.com/clickhouse-test-reports/51514/aa90e11241a97e0fd3f36c5ea0b57db50e299d2d/integration_tests__asan__analyzer__[4_6].html - https://s3.amazonaws.com/clickhouse-test-reports/51514/aa90e11241a97e0fd3f36c5ea0b57db50e299d2d/integration_tests__asan__analyzer__[5_6].html Signed-off-by: Azat Khuzhin --- tests/analyzer_integration_broken_tests.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 46b9ade43de..02f70c8a6df 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -196,3 +196,8 @@ test_quota/test.py::test_tracking_quota test_quota/test.py::test_users_xml_is_readonly test_replicating_constants/test.py::test_different_versions test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] +test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table] +test_backward_compatibility/test_data_skipping_indices.py::test_index +test_backward_compatibility/test_convert_ordinary.py::test_convert_ordinary_to_atomic +test_backward_compatibility/test_memory_bound_aggregation.py::test_backward_compatability +test_odbc_interaction/test.py::test_postgres_insert From 76abbce91829c8ec47b952d5d26a0d9df8ce2f85 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 16:33:04 +0000 Subject: [PATCH 305/473] Trying to add methods to queue --- src/Common/SystemLogBase.cpp | 102 ++++++++++++++++++++++----------- src/Common/SystemLogBase.h | 28 ++++----- src/Interpreters/SystemLog.cpp | 28 ++------- src/Interpreters/SystemLog.h | 4 +- 4 files changed, 85 insertions(+), 77 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 7373786e514..c979613b146 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -42,11 +42,8 @@ template SystemLogBase::SystemLogBase( const String & name_, std::shared_ptr> queue_) + : queue(queue_ ? queue_ : std::make_shared>(name_)) { - if (queue_) - queue = queue_; - else - queue = std::make_shared>(name_); } template @@ -153,27 +150,40 @@ void SystemLogQueue::add(const LogElement & element) } template -void SystemLogBase::add(const LogElement & element) +uint64_t SystemLogQueue::notifyFlush(bool force) { - queue->add(element); + uint64_t this_thread_requested_offset; + + { + std::lock_guard lock(mutex); + if (is_shutdown) + return uint64_t(-1); + + this_thread_requested_offset = queue_front_index + queue.size(); + + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + is_force_prepare_tables |= force; + requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + + flush_event.notify_all(); + } + + LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); + return this_thread_requested_offset; } template -void SystemLogBase::flush(bool force) +void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset_) { - uint64_t this_thread_requested_offset = notifyFlushImpl(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - // Use an arbitrary timeout to avoid endless waiting. 60s proved to be // too fast for our parallel functional tests, probably because they // heavily load the disk. const int timeout_seconds = 180; - std::unique_lock lock(queue->mutex); - bool result = queue->flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + std::unique_lock lock(mutex); + bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { - return flushed_up_to >= this_thread_requested_offset && !is_force_prepare_tables; + return flushed_up_to >= this_thread_requested_offset_ && !is_force_prepare_tables; }); if (!result) @@ -183,33 +193,57 @@ void SystemLogBase::flush(bool force) } } -template -void SystemLogBase::notifyFlush(bool force) { notifyFlushImpl(force); } +constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; template -uint64_t SystemLogBase::notifyFlushImpl(bool force) +void SystemLogQueue::pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) { - uint64_t this_thread_requested_offset; + std::unique_lock lock(queue->mutex); + flush_event.wait_for(lock, + std::chrono::milliseconds(flush_interval_milliseconds), + [&] () + { + return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; + } + ); - { - std::lock_guard lock(queue->mutex); - if (is_shutdown) - return uint64_t(-1); + queue_front_index += queue->size(); + to_flush_end = queue->queue_front_index; + // Swap with existing array from previous flush, to save memory + // allocations. + output.resize(0); + queue.swap(to_flush); - this_thread_requested_offset = queue->queue_front_index + queue->queue.size(); + should_prepare_tables_anyway = is_force_prepare_tables; - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= force; - queue->requested_flush_up_to = std::max(queue->requested_flush_up_to, this_thread_requested_offset); - - queue->flush_event.notify_all(); - } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; + exit_this_thread = is_shutdown; } +template +void SystemLogBase::add(const LogElement & element) +{ + queue->add(element); +} + +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } + +// template +// uint64_t SystemLogBase::notifyFlushImpl(bool force) +// { +// return queue->notifyFlush(force); +// } + #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 109334964d2..0c1f32672a9 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -81,6 +81,10 @@ public: //void push_back(const LogElement & element) { queue.push_back(element); } void shutdown() { is_shutdown = true; } + uint64_t notifyFlush(bool force); + void waitFlush(uint64_t this_thread_requested_offset_); + void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) + // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; // An always-incrementing index of the first message currently in the queue. @@ -96,12 +100,16 @@ public: // Requested to flush logs up to this index, exclusive uint64_t requested_flush_up_to = 0; - // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; - + // A flag that says we must create the tables even if the queue is empty. + bool is_force_prepare_tables = false; + + // Flushed log up to this index, exclusive + uint64_t flushed_up_to = 0; private: Poco::Logger * log; bool is_shutdown = false; + // Logged overflow message at this queue front index + uint64_t logged_queue_full_at_index = -1; }; template @@ -138,21 +146,7 @@ protected: std::shared_ptr> queue; - // A flag that says we must create the tables even if the queue is empty. - bool is_force_prepare_tables = false; - - // Flushed log up to this index, exclusive - uint64_t flushed_up_to = 0; - bool is_shutdown = false; - - // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; - -private: - uint64_t notifyFlushImpl(bool force); - - }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index b77cb2311d5..983c9f87c45 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -372,27 +372,7 @@ void SystemLog::savingThreadFunction() // Should we prepare table even if there are no new messages. bool should_prepare_tables_anyway = false; - { - std::unique_lock lock(queue->mutex); - queue->flush_event.wait_for(lock, - std::chrono::milliseconds(flush_interval_milliseconds), - [&] () - { - return queue->requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; - } - ); - - queue->queue_front_index += queue->size(); - to_flush_end = queue->queue_front_index; - // Swap with existing array from previous flush, to save memory - // allocations. - to_flush.resize(0); - queue->queue.swap(to_flush); - - should_prepare_tables_anyway = is_force_prepare_tables; - - exit_this_thread = is_shutdown; - } + queue->pop(to_flush, to_flush_end, should_prepare_tables_anyway, exit_this_thread); if (to_flush.empty()) { @@ -402,7 +382,7 @@ void SystemLog::savingThreadFunction() LOG_TRACE(log, "Table created (force)"); std::lock_guard lock(queue->mutex); - is_force_prepare_tables = false; + queue->is_force_prepare_tables = false; queue->flush_event.notify_all(); } } @@ -477,8 +457,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, { std::lock_guard lock(queue->mutex); - flushed_up_to = to_flush_end; - is_force_prepare_tables = false; + queue->flushed_up_to = to_flush_end; + queue->is_force_prepare_tables = false; queue->flush_event.notify_all(); } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 3f5ad8a5b6c..c730aa23319 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -121,9 +121,9 @@ protected: using Base::log; using Base::queue; // using Base::queue_front_index; - using Base::is_force_prepare_tables; + // using Base::is_force_prepare_tables; //using Base::requested_flush_up_to; - using Base::flushed_up_to; + // using Base::flushed_up_to; // using Base::logged_queue_full_at_index; private: From 918e3f7712a65554d4b5d0ae279c812aca3298d2 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 18:41:12 +0200 Subject: [PATCH 306/473] Fix flaky test --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/01076_parallel_alter_replicated_zookeeper.sh | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..7934ba7d8b5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1385,7 +1385,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) } const UInt64 parts_to_fetch_blocks = std::accumulate(parts_to_fetch.cbegin(), parts_to_fetch.cend(), 0, - [&](UInt64 acc, const String& part_name) + [&](UInt64 acc, const String & part_name) { if (const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version)) return acc + part_info->getBlocksCount(); diff --git a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh index 5f69427c0cd..79d2c736793 100755 --- a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh @@ -138,8 +138,13 @@ while true ; do done for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_mutate_mt_$i" + $CLICKHOUSE_CLIENT --query "CHECK TABLE concurrent_mutate_mt_$i" &> /dev/null # if we will remove something the output of select will be wrong $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_mutate_mt_$i" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE table='concurrent_mutate_mt_$i' and is_done=0" # all mutations have to be done $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE table='concurrent_mutate_mt_$i' and is_done=0" # for verbose output +done + +for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_mt_$i" done From b70349852c4f5b37a0c8b5db172b9ac6b901fb24 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 19:02:29 +0200 Subject: [PATCH 307/473] Better message on timeout and fix test --- src/Storages/IStorage.cpp | 19 ++++++++++++++----- src/Storages/IStorage.h | 1 + src/Storages/StorageMergeTree.cpp | 8 +++++++- .../02543_alter_rename_modify_stuck.sh | 15 +++------------ 4 files changed, 25 insertions(+), 18 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 8cf708acd8b..d4f796f71ca 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -71,15 +71,12 @@ TableLockHolder IStorage::tryLockForShare(const String & query_id, const std::ch return result; } -IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds & acquire_timeout) +std::optional IStorage::tryLockForAlter(const std::chrono::milliseconds & acquire_timeout) { AlterLockHolder lock{alter_lock, std::defer_lock}; if (!lock.try_lock_for(acquire_timeout)) - throw Exception(ErrorCodes::DEADLOCK_AVOIDED, - "Locking attempt for ALTER on \"{}\" has timed out! ({} ms) " - "Possible deadlock avoided. Client should retry.", - getStorageID().getFullTableName(), acquire_timeout.count()); + return {} if (is_dropped || is_detached) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID()); @@ -87,6 +84,18 @@ IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds return lock; } +IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds & acquire_timeout) +{ + + if (auto lock = tryLockForAlter(acquire_timeout); lock == std::nullopt) + throw Exception(ErrorCodes::DEADLOCK_AVOIDED, + "Locking attempt for ALTER on \"{}\" has timed out! ({} ms) " + "Possible deadlock avoided. Client should retry.", + getStorageID().getFullTableName(), acquire_timeout.count()); + else + return *lock; +} + TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b262d88db57..76641b656a2 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -283,6 +283,7 @@ public: /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. using AlterLockHolder = std::unique_lock; AlterLockHolder lockForAlter(const std::chrono::milliseconds & acquire_timeout); + std::optional tryLockForAlter(const std::chrono::milliseconds & acquire_timeout); /// Lock table exclusively. This lock must be acquired if you want to be /// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8cc2fbc2911..a20f9ae12fa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -603,7 +603,13 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer { /// It's important to serialize order of mutations with alter queries because /// they can depend on each other. - auto alter_lock = lockForAlter(query_context->getSettings().lock_acquire_timeout); + if (auto alter_lock = tryLockForAlter(query_context->getSettings().lock_acquire_timeout); alter_lock == std::nullopt) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Cannot start mutation in {}ms because some metadata-changing ALTER (MODIFY|RENAME|ADD|DROP) is currently executing. " + "You can change this timeout with `lock_acquire_timeout` setting", + query_context->getSettings().lock_acquire_timeout) + } version = startMutation(commands, query_context); } diff --git a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh index adaf1846552..8aa04c97937 100755 --- a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh +++ b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh @@ -34,19 +34,10 @@ done $CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename UPDATE v2 = 77 WHERE 1 = 1 SETTINGS mutations_sync = 2" & -counter=0 retries=60 - -I=0 -while [[ $counter -lt $retries ]]; do - I=$((I + 1)) - result=$($CLICKHOUSE_CLIENT --query "SELECT count() from system.mutations where database='${CLICKHOUSE_DATABASE}' and table='table_to_rename'") - if [[ $result == "2" ]]; then - break; - fi - sleep 0.1 - ((++counter)) -done +# we cannot wait in the same way like we do for previous alter +# because it's metadata alter and this one will wait for it +sleep 1 $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" From bfbf9f292efd5bc35952f91e920f8ae2bd85e5e2 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 19:03:04 +0200 Subject: [PATCH 308/473] Fix --- tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh index 8aa04c97937..1f517913076 100755 --- a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh +++ b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh @@ -37,7 +37,7 @@ $CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename UPDATE v2 = 77 WHERE 1 = # we cannot wait in the same way like we do for previous alter # because it's metadata alter and this one will wait for it -sleep 1 +sleep 3 $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" From 1dbc88630d56f2b815c2c5746f6706d2ff76a6bc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 17:09:13 +0000 Subject: [PATCH 309/473] Cosmetics --- cmake/limit_jobs.cmake | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 39a6f688488..acc38b6fa2a 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -1,7 +1,11 @@ +# Limit compiler/linker job concurrency to avoid OOMs on subtrees where compilation/linking is memory-intensive. +# # Usage from CMake: -# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte -# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte -# include (cmake/limit_jobs.cmake) +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte +# include (cmake/limit_jobs.cmake) +# +# (bigger values mean fewer jobs) cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) @@ -17,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + message(WARNING "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -28,7 +32,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + message(WARNING "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () @@ -43,7 +47,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs.") +message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB DRAM, 'OFF' means the native core count).") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From 2db14e36b4d187ed485891d862c8365a4ce7bf17 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 19:15:07 +0200 Subject: [PATCH 310/473] Fix build --- src/Storages/IStorage.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index d4f796f71ca..ae7659e074f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -76,7 +76,7 @@ std::optional IStorage::tryLockForAlter(const std::ch AlterLockHolder lock{alter_lock, std::defer_lock}; if (!lock.try_lock_for(acquire_timeout)) - return {} + return {}; if (is_dropped || is_detached) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID()); @@ -93,7 +93,7 @@ IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds "Possible deadlock avoided. Client should retry.", getStorageID().getFullTableName(), acquire_timeout.count()); else - return *lock; + return std::move(*lock); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a20f9ae12fa..353a647704e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -608,7 +608,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot start mutation in {}ms because some metadata-changing ALTER (MODIFY|RENAME|ADD|DROP) is currently executing. " "You can change this timeout with `lock_acquire_timeout` setting", - query_context->getSettings().lock_acquire_timeout) + query_context->getSettings().lock_acquire_timeout.totalMilliseconds()); } version = startMutation(commands, query_context); } From a955d5621d611dd1f1159a008141a8538575b6db Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jul 2023 18:53:12 +0000 Subject: [PATCH 311/473] Rename constants in KeyCondition analysis. --- src/Storages/MergeTree/KeyCondition.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f909d854cf6..3f02a6b197e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -564,7 +564,17 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } case (ActionsDAG::ActionType::COLUMN): { - res = &inverted_dag.addColumn({node.column, node.result_type, node.result_name}); + String name; + if (const auto * column_const = typeid_cast(node.column.get())) + /// Re-generate column name for constant. + /// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. + /// DAG from PK does not use it. This is breakig match by column name sometimes. + /// Ideally, we should not compare manes, but DAG subtrees instead. + name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName(); + else + name = node.result_name; + + res = &inverted_dag.addColumn({node.column, node.result_type, name}); break; } case (ActionsDAG::ActionType::ALIAS): From eb8629db5138b79cedc4096ecaa96c58e228d307 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 23:17:47 +0300 Subject: [PATCH 312/473] Update 02815_logical_error_cannot_get_column_name_of_set.sql --- .../02815_logical_error_cannot_get_column_name_of_set.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql index 98a8a8992b8..aa659165940 100644 --- a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql +++ b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql @@ -1,3 +1,3 @@ SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS } -SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION } -SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION } +SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } +SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } From 4043174a81f03d0ad272a5c5c5b3e1dc46a6968f Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 17 Jul 2023 23:46:45 +0200 Subject: [PATCH 313/473] fix special build --- src/Storages/MergeTree/MergeTreeDataWriter.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index d6c980d83d8..2fb6b1f22d4 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -52,7 +52,7 @@ public: * (split rows by partition) * Works deterministically: if same block was passed, function will return same result in same order. */ - static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr chunk_offsets = nullptr); + static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr async_insert_info = nullptr); /// This structure contains not completely written temporary part. /// Some writes may happen asynchronously, e.g. for blob storages. From e4f798fcda5bd912d9e5a14bb087fb32ea8a842e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:36:44 +0200 Subject: [PATCH 314/473] Remove useless header --- src/Common/DateLUTImpl.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index d7bdd0bb3d9..8146b35cc5f 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include From bc9bf009af1423a2528b169bbbdc13bc9bb04340 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 02:27:39 +0200 Subject: [PATCH 315/473] Inhibit settings randomization in the test `json_ghdata`. --- tests/queries/0_stateless/01825_type_json_ghdata.sh | 2 +- .../0_stateless/01825_type_json_ghdata_insert_select.sh | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01825_type_json_ghdata.sh b/tests/queries/0_stateless/01825_type_json_ghdata.sh index bdb439f756f..cea02131d86 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type 1 cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata FORMAT JSONAsObject" diff --git a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh index 487c95137ae..498e1db1f69 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh @@ -9,9 +9,9 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_string" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_from_string" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple()" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type 1 cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2 FORMAT JSONAsObject" cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_string FORMAT JSONAsString" From 182d8b8f081d73f07877ab1ce7269288b1cf70b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:36:44 +0200 Subject: [PATCH 316/473] Remove useless header --- src/Common/DateLUTImpl.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index d7bdd0bb3d9..8146b35cc5f 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include From 8a4f939e1af9337c99ade273b7ba4e992a7c6eea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 02:37:50 +0200 Subject: [PATCH 317/473] Slightly better diagnostics in tests --- tests/queries/0_stateless/02725_memory-for-merges.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02725_memory-for-merges.sql b/tests/queries/0_stateless/02725_memory-for-merges.sql index b6ae7af7f1a..347c8b2a8d3 100644 --- a/tests/queries/0_stateless/02725_memory-for-merges.sql +++ b/tests/queries/0_stateless/02725_memory-for-merges.sql @@ -22,6 +22,6 @@ OPTIMIZE TABLE 02725_memory_for_merges FINAL; SYSTEM FLUSH LOGS; WITH (SELECT uuid FROM system.tables WHERE table='02725_memory_for_merges' and database=currentDatabase()) as uuid -SELECT sum(peak_memory_usage) < 1024 * 1024 * 200 from system.part_log where table_uuid=uuid and event_type='MergeParts'; +SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where table_uuid=uuid and event_type='MergeParts'; DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; From d8dd480cd78834841a1173b67e1cdf44201898d7 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 17 Jul 2023 00:15:11 +0800 Subject: [PATCH 318/473] Disable implicit projections by default --- src/Core/Settings.h | 1 + src/Processors/QueryPlan/Optimizations/Optimizations.h | 2 +- .../Optimizations/QueryPlanOptimizationSettings.cpp | 1 + .../Optimizations/QueryPlanOptimizationSettings.h | 1 + .../QueryPlan/Optimizations/optimizeTree.cpp | 3 ++- .../Optimizations/optimizeUseAggregateProjection.cpp | 10 ++++++---- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- 7 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f304f0aab6..dcac91eaa75 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -577,6 +577,7 @@ class IColumn; M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ + M(Bool, optimize_use_implicit_projections, false, "Automatically choose implicit projections to perform SELECT query", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index bc47413cbb5..6ecec1359c5 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -111,7 +111,7 @@ void optimizePrimaryKeyCondition(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 21c7d362b17..e011fb8ecbe 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -19,6 +19,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.remove_redundant_distinct = from.query_plan_remove_redundant_distinct; settings.optimize_projection = from.optimize_use_projections && from.query_plan_optimize_projection; settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; + settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 967cfdaca7f..d98c34ce226 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -41,6 +41,7 @@ struct QueryPlanOptimizationSettings /// If reading from projection can be applied bool optimize_projection = false; bool force_use_projection = false; + bool optimize_use_implicit_projections = false; static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index e788918703e..6cb76d540f7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -126,7 +126,8 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes); + num_applied_projection + += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 6e76d6e67a7..f183bdca7a9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -433,7 +433,8 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( QueryPlan::Node & node, AggregatingStep & aggregating, ReadFromMergeTree & reading, - const std::shared_ptr & max_added_blocks) + const std::shared_ptr & max_added_blocks, + bool allow_implicit_projections) { const auto & keys = aggregating.getParams().keys; const auto & aggregates = aggregating.getParams().aggregates; @@ -453,7 +454,8 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (projection.type == ProjectionDescription::Type::Aggregate) agg_projections.push_back(&projection); - bool can_use_minmax_projection = metadata->minmax_count_projection && !reading.getMergeTreeData().has_lightweight_delete_parts.load(); + bool can_use_minmax_projection = allow_implicit_projections && metadata->minmax_count_projection + && !reading.getMergeTreeData().has_lightweight_delete_parts.load(); if (!can_use_minmax_projection && agg_projections.empty()) return candidates; @@ -543,7 +545,7 @@ static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) return nullptr; } -bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections) { if (node.children.size() != 1) return false; @@ -568,7 +570,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); - auto candidates = getAggregateProjectionCandidates(node, *aggregating, *reading, max_added_blocks); + auto candidates = getAggregateProjectionCandidates(node, *aggregating, *reading, max_added_blocks, allow_implicit_projections); AggregateProjectionCandidate * best_candidate = nullptr; if (candidates.minmax_projection) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ec8ce3f5e3d..9c5e45aa488 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6994,7 +6994,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg ProjectionCandidate * selected_candidate = nullptr; size_t min_sum_marks = std::numeric_limits::max(); - if (metadata_snapshot->minmax_count_projection && !has_lightweight_delete_parts.load(std::memory_order_relaxed)) /// Disable ReadFromStorage for parts with lightweight. + if (settings.optimize_use_implicit_projections && metadata_snapshot->minmax_count_projection + && !has_lightweight_delete_parts.load(std::memory_order_relaxed)) /// Disable ReadFromStorage for parts with lightweight. add_projection_candidate(*metadata_snapshot->minmax_count_projection, true); std::optional minmax_count_projection_candidate; if (!candidates.empty()) From 796a92d9ab49298c208b3809f20640f87ac3a08d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 17 Jul 2023 10:17:33 +0800 Subject: [PATCH 319/473] Fix tests --- .../01505_trivial_count_with_partition_predicate.sql | 1 + tests/queries/0_stateless/01710_minmax_count_projection.sql | 2 +- tests/queries/0_stateless/01739_index_hint.reference | 2 +- tests/queries/0_stateless/01739_index_hint.sql | 2 +- .../0_stateless/02680_illegal_type_of_filter_projection.sql | 2 +- 5 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql index 030db421683..aaf88f95f0c 100644 --- a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql +++ b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql @@ -6,6 +6,7 @@ create table test1(p DateTime, k int) engine MergeTree partition by toDate(p) or insert into test1 values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); set max_rows_to_read = 1; +set optimize_use_implicit_projections = 1; -- non-optimized select count() from test1 settings max_parallel_replicas = 3; -- optimized (toYear is monotonic and we provide the partition expr as is) diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index c17f0e1e1fb..bc8327e3631 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -4,7 +4,7 @@ create table d (i int, j int) engine MergeTree partition by i % 2 order by tuple insert into d select number, number from numbers(10000); -set max_rows_to_read = 2, optimize_use_projections = 1; +set max_rows_to_read = 2, optimize_use_projections = 1, optimize_use_implicit_projections = 1; select min(i), max(i), count() from d; select min(i), max(i), count() from d group by _partition_id order by _partition_id; diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 71dfab29154..3a4b380de65 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -30,6 +30,6 @@ SELECT sum(t) FROM XXXX WHERE indexHint(t = 42); drop table if exists XXXX; create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192; insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; 100000 drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index 30dfa43d334..e1e66c630e1 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -30,6 +30,6 @@ create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings inde insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; drop table XXXX; diff --git a/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql index 3ef3b8a4fe6..d20e4deee27 100644 --- a/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql +++ b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql @@ -1,3 +1,3 @@ CREATE TABLE test_tuple (`p` DateTime, `i` int, `j` int) ENGINE = MergeTree PARTITION BY (toDate(p), i) ORDER BY j SETTINGS index_granularity = 1; insert into test_tuple values (1, 1, 1); -SELECT count() FROM test_tuple PREWHERE sipHash64(sipHash64(p, toString(toDate(p))), toString(toDate(p))) % -0. WHERE i > NULL settings optimize_trivial_count_query=0; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT count() FROM test_tuple PREWHERE sipHash64(sipHash64(p, toString(toDate(p))), toString(toDate(p))) % -0. WHERE i > NULL settings optimize_trivial_count_query=0, optimize_use_implicit_projections=1; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } From ef521fd6ce79798a2661bce13050d4bc350ecc41 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 17 Jul 2023 14:36:18 +0800 Subject: [PATCH 320/473] Fix more tests --- tests/queries/0_stateless/01848_partition_value_column.sql | 2 ++ .../02675_profile_events_from_query_log_and_client.reference | 4 ++-- .../02675_profile_events_from_query_log_and_client.sh | 4 ++-- tests/queries/1_stateful/00172_early_constant_folding.sql | 1 + 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01848_partition_value_column.sql b/tests/queries/0_stateless/01848_partition_value_column.sql index d13e4508789..28d842af3e9 100644 --- a/tests/queries/0_stateless/01848_partition_value_column.sql +++ b/tests/queries/0_stateless/01848_partition_value_column.sql @@ -5,6 +5,8 @@ create table tbl(dt DateTime, i int, j String, v Float64) engine MergeTree parti insert into tbl values ('2021-04-01 00:01:02', 1, '123', 4), ('2021-04-01 01:01:02', 1, '12', 4), ('2021-04-01 02:11:02', 2, '345', 4), ('2021-04-01 04:31:02', 2, '2', 4), ('2021-04-02 00:01:02', 1, '1234', 4), ('2021-04-02 00:01:02', 2, '123', 4), ('2021-04-02 00:01:02', 3, '12', 4), ('2021-04-02 00:01:02', 4, '1', 4); +set optimize_use_implicit_projections = 1; + select count() from tbl where _partition_value = ('2021-04-01', 1, 2) settings max_rows_to_read = 1; select count() from tbl where _partition_value.1 = '2021-04-01' settings max_rows_to_read = 4; select count() from tbl where _partition_value.2 = 0 settings max_rows_to_read = 4; diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index 3f34d5e2c79..a97879eaca8 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -17,7 +17,7 @@ INSERT and READ INSERT DROP CHECK with query_log QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; FileOpen 8 -QueryFinish SELECT \'1\', min(t) FROM times; FileOpen 0 +QueryFinish SELECT \'1\', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; FileOpen 0 QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; FileOpen 8 -QueryFinish SELECT \'2\', min(t) FROM times; FileOpen 0 +QueryFinish SELECT \'2\', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; FileOpen 0 QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; FileOpen 8 diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index adc9525ef81..288f1129b53 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -44,13 +44,13 @@ INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; echo "READ" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " -SELECT '1', min(t) FROM times; +SELECT '1', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' echo "INSERT and READ INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; -SELECT '2', min(t) FROM times; +SELECT '2', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' diff --git a/tests/queries/1_stateful/00172_early_constant_folding.sql b/tests/queries/1_stateful/00172_early_constant_folding.sql index 1ed7b8719b4..19f99f107ac 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.sql +++ b/tests/queries/1_stateful/00172_early_constant_folding.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel-replicas set max_threads=10; +set optimize_use_implicit_projections=1; EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1; From c782689c2d64e0a3bfdc97bcd4c9f53a00622d7f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 18 Jul 2023 09:49:45 +0800 Subject: [PATCH 321/473] added to compatibility setting --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 2886cdd288d..3e58750e1d2 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,6 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.7", {{"optimize_use_implicit_projections", true, false, "Disable implicit projections due to unexpected results."}}}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, From 56a5446af3958d37c0f2cbc8e879beb3876b8857 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 18 Jul 2023 10:28:51 +0800 Subject: [PATCH 322/473] format Signed-off-by: Lloyd-Pottiger --- docs/en/sql-reference/functions/date-time-functions.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 9a42c3ec835..66e4f6bc9eb 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1130,10 +1130,12 @@ Result: └──────────────────────────────────────────────────────────────┘ ``` -## now/current_timestamp +## now Returns the current date and time at the moment of query analysis. The function is a constant expression. +Alias: `current_timestamp`. + **Syntax** ``` sql @@ -1259,11 +1261,13 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today/curdate/current_date +## today Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. +Aliases: `curdate`, `current_date`. + ## yesterday Accepts zero arguments and returns yesterday’s date at one of the moments of query analysis. From f3dcf4db3a30f552932aa08b72116935de9d519d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 18 Jul 2023 11:00:39 +0800 Subject: [PATCH 323/473] update 02815_range_dict_no_direct_join.sql --- tests/queries/0_stateless/02815_range_dict_no_direct_join.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql index 249ffdf8089..f4e5ccccf78 100644 --- a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql @@ -30,4 +30,4 @@ RANGE(MIN discount_start_date MAX discount_end_date); CREATE TABLE ids (id UInt64) ENGINE = Memory; INSERT INTO ids SELECT * FROM numbers(10); -SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id SETTINGS join_algorithm = 'direct'; \ No newline at end of file +SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS join_algorithm = 'direct'; From 9b7df9e12e9e85e1a2ecb2788c56dff74b75b309 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 18 Jul 2023 05:51:55 +0000 Subject: [PATCH 324/473] fix for analyzer --- src/Interpreters/TableJoin.h | 4 ++-- src/Planner/PlannerJoins.cpp | 3 ++- .../0_stateless/02815_range_dict_no_direct_join.reference | 6 ++++++ .../queries/0_stateless/02815_range_dict_no_direct_join.sql | 1 + 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index ba3befab59b..5d14a57759f 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -223,10 +223,10 @@ public: { /// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm. /// It's behaviour that was initially supported by clickhouse. - bool is_enbaled_by_default = val == JoinAlgorithm::DEFAULT + bool is_enabled_by_default = val == JoinAlgorithm::DEFAULT || val == JoinAlgorithm::HASH || val == JoinAlgorithm::DIRECT; - if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enbaled_by_default) + if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enabled_by_default) return true; return join_algorithm.isSet(val); } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 7da10a8523b..e495b0967e9 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -542,7 +542,8 @@ void trySetStorageInTableJoin(const QueryTreeNodePtr & table_expression, std::sh if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) return; - if (auto storage_dictionary = std::dynamic_pointer_cast(storage); storage_dictionary) + if (auto storage_dictionary = std::dynamic_pointer_cast(storage); + storage_dictionary && storage_dictionary->getDictionary()->getSpecialKeyType() != DictionarySpecialKeyType::Range) table_join->setStorageJoin(std::dynamic_pointer_cast(storage_dictionary->getDictionary())); else if (auto storage_key_value = std::dynamic_pointer_cast(storage); storage_key_value) table_join->setStorageJoin(storage_key_value); diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference index 3576991f7a4..051716a791e 100644 --- a/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference @@ -4,3 +4,9 @@ 2 0.4 3 0.5 3 0.6 +1 0.1 +1 0.2 +2 0.3 +2 0.4 +3 0.5 +3 0.6 diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql index f4e5ccccf78..e3af53fa335 100644 --- a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql @@ -31,3 +31,4 @@ CREATE TABLE ids (id UInt64) ENGINE = Memory; INSERT INTO ids SELECT * FROM numbers(10); SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS join_algorithm = 'direct'; +SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS allow_experimental_analyzer = 1; From 2f6bf4590b465e84641f090cfa1ee80c3cb28602 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 7 Jul 2023 17:13:41 +0200 Subject: [PATCH 325/473] Filter databases list before querying potentially slow fields --- .../System/StorageSystemDatabases.cpp | 38 ++++++++++++++++++- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index a3d05281b28..ee8a1de9b24 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include @@ -69,20 +71,52 @@ static String getEngineFull(const ContextPtr & ctx, const DatabasePtr & database return engine_full; } -void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +static ColumnPtr getFilteredDatabases(const Databases & databases, const SelectQueryInfo & query_info, ContextPtr context) +{ + MutableColumnPtr name_column = ColumnString::create(); + MutableColumnPtr engine_column = ColumnString::create(); + MutableColumnPtr uuid_column = ColumnUUID::create(); + + for (const auto & [database_name, database] : databases) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + name_column->insert(database_name); + engine_column->insert(database->getEngineName()); + uuid_column->insert(database->getUUID()); + } + + Block block + { + ColumnWithTypeAndName(std::move(name_column), std::make_shared(), "name"), + ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine"), + ColumnWithTypeAndName(std::move(uuid_column), std::make_shared(), "uuid") + }; + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block, context); + return block.getByPosition(0).column; +} + +void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES); const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & [database_name, database] : databases) + ColumnPtr filtered_databases_column = getFilteredDatabases(databases, query_info, context); + + for (size_t i = 0; i < filtered_databases_column->size(); ++i) { + auto database_name = filtered_databases_column->getDataAt(i).toString(); + if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name)) continue; if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) continue; /// filter out the internal database for temporary tables in system.databases, asynchronous metric "NumberOfDatabases" behaves the same way + const auto database = databases.at(database_name); + res_columns[0]->insert(database_name); res_columns[1]->insert(database->getEngineName()); res_columns[2]->insert(context->getPath() + database->getDataPath()); From 44595b98d0c7bd25bd92da69d4b80907d81b25db Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 17 Jul 2023 21:20:33 +0200 Subject: [PATCH 326/473] Update src/Storages/System/StorageSystemDatabases.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Storages/System/StorageSystemDatabases.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index ee8a1de9b24..2fcc91e49bb 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -115,7 +115,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) continue; /// filter out the internal database for temporary tables in system.databases, asynchronous metric "NumberOfDatabases" behaves the same way - const auto database = databases.at(database_name); + const auto & database = databases.at(database_name); res_columns[0]->insert(database_name); res_columns[1]->insert(database->getEngineName()); From 562ea03ce70c5fe0b44298b18f898e87d9ab44a5 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Mon, 17 Jul 2023 12:54:34 +0300 Subject: [PATCH 327/473] FunctionsURL.h to StringHelpers.h and move out of URL/ --- src/Functions/{URL/FunctionsURL.h => StringHelpers.h} | 8 ++++---- src/Functions/URL/FirstSignificantSubdomainCustomImpl.h | 2 +- src/Functions/URL/basename.cpp | 2 +- src/Functions/URL/fragment.h | 2 +- src/Functions/URL/netloc.cpp | 3 +-- src/Functions/URL/path.cpp | 2 +- src/Functions/URL/path.h | 2 +- src/Functions/URL/pathFull.cpp | 2 +- src/Functions/URL/protocol.h | 3 +-- src/Functions/URL/queryString.h | 2 +- src/Functions/URL/queryStringAndFragment.h | 3 +-- 11 files changed, 14 insertions(+), 17 deletions(-) rename src/Functions/{URL/FunctionsURL.h => StringHelpers.h} (93%) diff --git a/src/Functions/URL/FunctionsURL.h b/src/Functions/StringHelpers.h similarity index 93% rename from src/Functions/URL/FunctionsURL.h rename to src/Functions/StringHelpers.h index 362042e31e1..a0f4d18aa80 100644 --- a/src/Functions/URL/FunctionsURL.h +++ b/src/Functions/StringHelpers.h @@ -7,8 +7,8 @@ namespace DB { -/** URL processing functions. See implementation in separate .cpp files. - * All functions are not strictly follow RFC, instead they are maximally simplified for performance reasons. +/** These helpers are used by URL processing functions. See implementation in separate .cpp files. + * All functions do not strictly follow RFC, instead they are maximally simplified for performance reasons. * * Functions for extraction parts of URL. * If URL has nothing like, then empty string is returned. @@ -101,7 +101,7 @@ struct ExtractSubstringImpl static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by URL functions"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by this function"); } }; @@ -156,7 +156,7 @@ struct CutSubstringImpl static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by URL functions"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported this function"); } }; diff --git a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h index 5862265ce7d..93691e35741 100644 --- a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h +++ b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h @@ -1,8 +1,8 @@ #pragma once #include -#include #include +#include #include #include #include diff --git a/src/Functions/URL/basename.cpp b/src/Functions/URL/basename.cpp index 6992f924ef2..bc747e2595f 100644 --- a/src/Functions/URL/basename.cpp +++ b/src/Functions/URL/basename.cpp @@ -1,7 +1,7 @@ #include #include +#include #include -#include "FunctionsURL.h" namespace DB { diff --git a/src/Functions/URL/fragment.h b/src/Functions/URL/fragment.h index 6c11d0fbb34..0414c4f64a2 100644 --- a/src/Functions/URL/fragment.h +++ b/src/Functions/URL/fragment.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB { diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index bc34e34a40d..bf3e8a471ef 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB @@ -154,4 +154,3 @@ REGISTER_FUNCTION(Netloc) } } - diff --git a/src/Functions/URL/path.cpp b/src/Functions/URL/path.cpp index ccc7dedb724..8d609f43191 100644 --- a/src/Functions/URL/path.cpp +++ b/src/Functions/URL/path.cpp @@ -1,6 +1,6 @@ #include #include -#include "FunctionsURL.h" +#include #include "path.h" #include diff --git a/src/Functions/URL/path.h b/src/Functions/URL/path.h index 89244659088..a0dd5eea2d3 100644 --- a/src/Functions/URL/path.h +++ b/src/Functions/URL/path.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Functions/URL/pathFull.cpp b/src/Functions/URL/pathFull.cpp index 002770e812a..9aacee21fed 100644 --- a/src/Functions/URL/pathFull.cpp +++ b/src/Functions/URL/pathFull.cpp @@ -1,6 +1,6 @@ #include #include -#include "FunctionsURL.h" +#include #include "path.h" #include diff --git a/src/Functions/URL/protocol.h b/src/Functions/URL/protocol.h index 74c0bb820b4..c1d83192835 100644 --- a/src/Functions/URL/protocol.h +++ b/src/Functions/URL/protocol.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB @@ -54,4 +54,3 @@ struct ExtractProtocol }; } - diff --git a/src/Functions/URL/queryString.h b/src/Functions/URL/queryString.h index a0777a5c9a1..603450d102b 100644 --- a/src/Functions/URL/queryString.h +++ b/src/Functions/URL/queryString.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB diff --git a/src/Functions/URL/queryStringAndFragment.h b/src/Functions/URL/queryStringAndFragment.h index ed19cd14b74..27008388e4d 100644 --- a/src/Functions/URL/queryStringAndFragment.h +++ b/src/Functions/URL/queryStringAndFragment.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB @@ -34,4 +34,3 @@ struct ExtractQueryStringAndFragment }; } - From 14ab1f2a98ad50ea08d8e089d4c426c353bd686b Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 12 Jul 2023 10:18:39 +0200 Subject: [PATCH 328/473] Fix SipHash doc Changes originally made by Robert Schulze (@rschu1ze) --- docs/en/sql-reference/functions/hash-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 06097d92480..556fe622c27 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 (#hash_functions-siphash64) +## sipHash64 {#hash_functions-siphash64} Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** From 693892801ce2a56dac7a55694e90e216ee3c5bcc Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 17 Jul 2023 10:18:32 +0200 Subject: [PATCH 329/473] Re-add keyed SipHash and fix issues with current keyed hashes framwork --- src/Functions/FunctionsHashing.h | 289 +++++++++++++++++++------ src/Functions/FunctionsHashingMisc.cpp | 5 + 2 files changed, 222 insertions(+), 72 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 279294b367c..20612e91a62 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -79,28 +79,49 @@ namespace impl UInt64 key1 = 0; }; - static SipHashKey parseSipHashKey(const ColumnWithTypeAndName & key) + struct SipHashKeyColumns { - SipHashKey ret{}; + ColumnPtr key0; + ColumnPtr key1; + bool is_const; - const auto * tuple = checkAndGetColumn(key.column.get()); + size_t size() const + { + assert(key0 && key1); + assert(key0->size() == key1->size()); + return key0->size(); + } + SipHashKey getKey(size_t i) const + { + if (is_const) + i = 0; + return {key0->get64(i), key1->get64(i)}; + } + }; + + static SipHashKeyColumns parseSipHashKeyColumns(const ColumnWithTypeAndName & key) + { + const ColumnTuple * tuple = nullptr; + const auto * column = key.column.get(); + bool is_const = false; + if (isColumnConst(*column)) + { + is_const = true; + tuple = checkAndGetColumnConstData(column); + } + else + tuple = checkAndGetColumn(column); if (!tuple) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple"); - if (tuple->tupleSize() != 2) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); - if (tuple->empty()) - return ret; - - if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) - ret.key0 = key0col->get64(0); - else + SipHashKeyColumns ret{tuple->getColumnPtr(0), tuple->getColumnPtr(1), is_const}; + assert(ret.key0); + if (!checkColumn(*ret.key0)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); - - if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) - ret.key1 = key1col->get64(0); - else + assert(ret.key1); + if (!checkColumn(*ret.key1)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); return ret; @@ -329,8 +350,10 @@ struct SipHash64KeyedImpl static constexpr auto name = "sipHash64Keyed"; using ReturnType = UInt64; using Key = impl::SipHashKey; + using KeyColumns = impl::SipHashKeyColumns; - static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } + static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } + static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } static UInt64 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash64Keyed(key.key0, key.key1, begin, size); } @@ -371,8 +394,10 @@ struct SipHash128KeyedImpl static constexpr auto name = "sipHash128Keyed"; using ReturnType = UInt128; using Key = impl::SipHashKey; + using KeyColumns = impl::SipHashKeyColumns; - static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } + static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } + static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash128Keyed(key.key0, key.key1, begin, size); } @@ -405,6 +430,36 @@ struct SipHash128ReferenceImpl static constexpr bool use_int_hash_for_pods = false; }; +struct SipHash128ReferenceKeyedImpl +{ + static constexpr auto name = "sipHash128ReferenceKeyed"; + using ReturnType = UInt128; + using Key = impl::SipHashKey; + using KeyColumns = impl::SipHashKeyColumns; + + static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } + static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } + + static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) + { + return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size); + } + + static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) + { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + UInt128 tmp; + reverseMemcpy(&tmp, &h1, sizeof(UInt128)); + h1 = tmp; + reverseMemcpy(&tmp, &h2, sizeof(UInt128)); + h2 = tmp; +#endif + UInt128 hashes[] = {h1, h2}; + return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); + } + + static constexpr bool use_int_hash_for_pods = false; +}; /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. @@ -1023,7 +1078,7 @@ private: DECLARE_MULTITARGET_CODE( -template +template class FunctionAnyHash : public IFunction { public: @@ -1033,9 +1088,12 @@ private: using ToType = typename Impl::ReturnType; template - void executeIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1044,6 +1102,9 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); if constexpr (Impl::use_int_hash_for_pods) { @@ -1077,6 +1138,14 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { + if constexpr (Keyed) + { + if (!key_cols.is_const) + { + ColumnPtr full_column = col_from_const->convertToFullColumn(); + return executeIntType(key_cols, full_column.get(), vec_to); + } + } auto value = col_from_const->template getValue(); ToType hash; @@ -1107,8 +1176,15 @@ private: if constexpr (first) vec_to.assign(size, hash); else + { for (size_t i = 0; i < size; ++i) + { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); + } + } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1116,9 +1192,12 @@ private: } template - void executeBigIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeBigIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1127,6 +1206,9 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); if constexpr (std::endian::native == std::endian::little) hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); else @@ -1143,6 +1225,14 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { + if constexpr (Keyed) + { + if (!key_cols.is_const) + { + ColumnPtr full_column = col_from_const->convertToFullColumn(); + return executeBigIntType(key_cols, full_column.get(), vec_to); + } + } auto value = col_from_const->template getValue(); ToType hash; @@ -1158,8 +1248,15 @@ private: if constexpr (first) vec_to.assign(size, hash); else + { for (size_t i = 0; i < size; ++i) + { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); + } + } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1167,10 +1264,16 @@ private: } template - void executeGeneric(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeGeneric(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); for (size_t i = 0, size = column->size(); i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); StringRef bytes = column->getDataAt(i); const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) @@ -1181,8 +1284,11 @@ private: } template - void executeString(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeString(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); if (const ColumnString * col_from = checkAndGetColumn(column)) { const typename ColumnString::Chars & data = col_from->getChars(); @@ -1192,6 +1298,9 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); @@ -1212,6 +1321,9 @@ private: for (size_t i = 0; i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) vec_to[i] = hash; @@ -1221,6 +1333,14 @@ private: } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) { + if constexpr (Keyed) + { + if (!key_cols.is_const) + { + ColumnPtr full_column = col_from_const->convertToFullColumn(); + return executeString(key_cols, full_column.get(), vec_to); + } + } String value = col_from_const->getValue(); const ToType hash = apply(key, value.data(), value.size()); const size_t size = vec_to.size(); @@ -1228,8 +1348,15 @@ private: if constexpr (first) vec_to.assign(size, hash); else + { for (size_t i = 0; i < size; ++i) + { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); + } + } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1237,7 +1364,7 @@ private: } template - void executeArray(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeArray(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const { const IDataType * nested_type = typeid_cast(*type).getNestedType().get(); @@ -1249,13 +1376,19 @@ private: typename ColumnVector::Container vec_temp(nested_size); bool nested_is_first = true; - executeForArgument(key, nested_type, nested_column, vec_temp, nested_is_first); + executeForArgument(key_cols, nested_type, nested_column, vec_temp, nested_is_first); const size_t size = offsets.size(); ColumnArray::Offset current_offset = 0; + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); for (size_t i = 0; i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); ColumnArray::Offset next_offset = offsets[i]; ToType hash; @@ -1279,7 +1412,7 @@ private: { /// NOTE: here, of course, you can do without the materialization of the column. ColumnPtr full_column = col_from_const->convertToFullColumn(); - executeArray(key, type, full_column.get(), vec_to); + executeArray(key_cols, type, full_column.get(), vec_to); } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1287,7 +1420,7 @@ private: } template - void executeAny(const KeyType & key, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const + void executeAny(const KeyColumnsType & key_cols, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const { WhichDataType which(from_type); @@ -1295,40 +1428,45 @@ private: throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", icolumn->getName(), icolumn->size(), vec_to.size(), getName()); - if (which.isUInt8()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt16()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt32()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt64()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt128()) executeBigIntType(key, icolumn, vec_to); - else if (which.isUInt256()) executeBigIntType(key, icolumn, vec_to); - else if (which.isInt8()) executeIntType(key, icolumn, vec_to); - else if (which.isInt16()) executeIntType(key, icolumn, vec_to); - else if (which.isInt32()) executeIntType(key, icolumn, vec_to); - else if (which.isInt64()) executeIntType(key, icolumn, vec_to); - else if (which.isInt128()) executeBigIntType(key, icolumn, vec_to); - else if (which.isInt256()) executeBigIntType(key, icolumn, vec_to); - else if (which.isUUID()) executeBigIntType(key, icolumn, vec_to); - else if (which.isIPv4()) executeIntType(key, icolumn, vec_to); - else if (which.isIPv6()) executeBigIntType(key, icolumn, vec_to); - else if (which.isEnum8()) executeIntType(key, icolumn, vec_to); - else if (which.isEnum16()) executeIntType(key, icolumn, vec_to); - else if (which.isDate()) executeIntType(key, icolumn, vec_to); - else if (which.isDate32()) executeIntType(key, icolumn, vec_to); - else if (which.isDateTime()) executeIntType(key, icolumn, vec_to); + if constexpr (Keyed) + if ((!key_cols.is_const && key_cols.size() != vec_to.size()) + || (key_cols.is_const && key_cols.size() != 1)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key column size {} doesn't match result column size {} of function {}", key_cols.size(), vec_to.size(), getName()); + + if (which.isUInt8()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt16()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt64()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt128()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isUInt256()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isInt8()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt16()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt64()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt128()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isInt256()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isUUID()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isIPv4()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isIPv6()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isEnum8()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isEnum16()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isDate()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isDate32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isDateTime()) executeIntType(key_cols, icolumn, vec_to); /// TODO: executeIntType() for Decimal32/64 leads to incompatible result - else if (which.isDecimal32()) executeBigIntType(key, icolumn, vec_to); - else if (which.isDecimal64()) executeBigIntType(key, icolumn, vec_to); - else if (which.isDecimal128()) executeBigIntType(key, icolumn, vec_to); - else if (which.isDecimal256()) executeBigIntType(key, icolumn, vec_to); - else if (which.isFloat32()) executeIntType(key, icolumn, vec_to); - else if (which.isFloat64()) executeIntType(key, icolumn, vec_to); - else if (which.isString()) executeString(key, icolumn, vec_to); - else if (which.isFixedString()) executeString(key, icolumn, vec_to); - else if (which.isArray()) executeArray(key, from_type, icolumn, vec_to); - else executeGeneric(key, icolumn, vec_to); + else if (which.isDecimal32()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isDecimal64()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isDecimal128()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isDecimal256()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isFloat32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isFloat64()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isString()) executeString(key_cols, icolumn, vec_to); + else if (which.isFixedString()) executeString(key_cols, icolumn, vec_to); + else if (which.isArray()) executeArray(key_cols, from_type, icolumn, vec_to); + else executeGeneric(key_cols, icolumn, vec_to); } - void executeForArgument(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const + void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. if (const ColumnTuple * tuple = typeid_cast(column)) @@ -1337,7 +1475,7 @@ private: const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); for (size_t i = 0; i < tuple_size; ++i) - executeForArgument(key, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); + executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData(column)) { @@ -1347,24 +1485,24 @@ private: for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); - executeForArgument(key, tuple_types[i].get(), tmp.get(), vec_to, is_first); + executeForArgument(key_cols, tuple_types[i].get(), tmp.get(), vec_to, is_first); } } else if (const auto * map = checkAndGetColumn(column)) { const auto & type_map = assert_cast(*type); - executeForArgument(key, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); + executeForArgument(key_cols, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); } else if (const auto * const_map = checkAndGetColumnConst(column)) { - executeForArgument(key, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); + executeForArgument(key_cols, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); } else { if (is_first) - executeAny(key, type, column, vec_to); + executeAny(key_cols, type, column, vec_to); else - executeAny(key, type, column, vec_to); + executeAny(key_cols, type, column, vec_to); } is_first = false; @@ -1395,6 +1533,9 @@ public: { auto col_to = ColumnVector::create(input_rows_count); + if (input_rows_count == 0) + return col_to; + typename ColumnVector::Container & vec_to = col_to->getData(); /// If using a "keyed" algorithm, the first argument is the key and @@ -1408,17 +1549,17 @@ public: vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); } - KeyType key{}; + KeyColumnsType key_cols{}; if constexpr (Keyed) if (!arguments.empty()) - key = Impl::parseKey(arguments[0]); + key_cols = Impl::parseKeyColumns(arguments[0]); /// The function supports arbitrary number of arguments of arbitrary types. bool is_first_argument = true; for (size_t i = first_data_argument; i < arguments.size(); ++i) { const auto & col = arguments[i]; - executeForArgument(key, col.type.get(), col.column.get(), vec_to, is_first_argument); + executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); } if constexpr (std::is_same_v) /// backward-compatible @@ -1450,17 +1591,19 @@ public: ) // DECLARE_MULTITARGET_CODE -template -class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash +template +class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash { public: explicit FunctionAnyHash(ContextPtr context) : selector(context) { - selector.registerImplementation>(); + selector + .registerImplementation>(); #if USE_MULTITARGET_CODE - selector.registerImplementation>(); - selector.registerImplementation>(); + selector.registerImplementation>(); + selector + .registerImplementation>(); #endif } @@ -1696,7 +1839,7 @@ struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; using FunctionSipHash64 = FunctionAnyHash; -using FunctionSipHash64Keyed = FunctionAnyHash; +using FunctionSipHash64Keyed = FunctionAnyHash; using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL @@ -1710,8 +1853,10 @@ using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; -using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; +using FunctionSipHash128ReferenceKeyed + = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index 56c3c1ed00c..f56568b2508 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -20,6 +20,11 @@ REGISTER_FUNCTION(Hashing) .examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}}, .categories{"Hash"} }); + factory.registerFunction(FunctionDocumentation{ + .description = "Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument " + "instead of using a fixed key.", + .examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}}, + .categories{"Hash"}}); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); From fb23ece79730545debd90ca2032279f243ea1208 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 12 Jul 2023 10:24:55 +0200 Subject: [PATCH 330/473] Add tests for memsan bug Changes originally made by Robert Schulze (@rschu1ze) --- tests/queries/0_stateless/02534_keyed_siphash.reference | 5 +++++ tests/queries/0_stateless/02534_keyed_siphash.sql | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index ccc514e7ea2..697b81a1390 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -197,3 +197,8 @@ E28DBDE7FE22E41C Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 +Check memsan bug +1CE422FEE7BD8DE20000000000000000 +7766709361750702608 +20AF99D3A87829E0 +12489502208762728797 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 900b99f548a..bc7dda92e2a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -281,3 +281,9 @@ INSERT INTO tab VALUES ((2, 2), 4); SELECT sipHash64Keyed(key, val) FROM tab; SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; DROP TABLE tab; + +SELECT 'Check memsan bug'; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); +SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); +SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); +SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); From 425d9c17c7501b7ad7f9302312af59e6d5a84c95 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 17 Jul 2023 10:15:19 +0200 Subject: [PATCH 331/473] Fix memsan test --- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 697b81a1390..4f6978e7ec7 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -199,6 +199,6 @@ Check bug with hashing of const integer values 11862823756610506724 Check memsan bug 1CE422FEE7BD8DE20000000000000000 -7766709361750702608 +18096612095653370192 20AF99D3A87829E0 12489502208762728797 From e61426f7ae9295ffa2801a8a0bfb754347c9cb86 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 17 Jul 2023 10:15:54 +0200 Subject: [PATCH 332/473] Add keyed SipHash tests --- .../0_stateless/02534_keyed_siphash.reference | 21 ++ .../0_stateless/02534_keyed_siphash.sql | 65 ++++++ .../02552_siphash128_reference.reference | 129 +++++++++++ .../02552_siphash128_reference.sql | 206 ++++++++++++++++++ 4 files changed, 421 insertions(+) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 4f6978e7ec7..ea77e5c3963 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -202,3 +202,24 @@ Check memsan bug 18096612095653370192 20AF99D3A87829E0 12489502208762728797 +Check const columns +15080046610211022027 +15080046610211022027 +15080046610211022027 +15080046610211022027 +2E779C73D13981AA1AE19AFF9617EA49 +2E779C73D13981AA1AE19AFF9617EA49 +Check multiple keys +11862823756610506724 +9357996107237883963 +Check multiple keys 2 +11862823756610506724 +9357996107237883963 +11862823756610506724 +11862823756610506724 +Check multiple keys 3 +11862823756610506724 +9357996107237883963 +- +11862823756610506724 +11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index bc7dda92e2a..70ec8470ca3 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -287,3 +287,68 @@ SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807) SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); + +SELECT 'Check const columns'; +DROP TABLE IF EXISTS sipHashKeyed_test; + +CREATE TABLE sipHashKeyed_test +ENGINE = Memory() +AS +SELECT +1 a, +'test' b; + +SELECT +sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); + +SELECT +sipHash64(tuple(*)) +FROM +sipHashKeyed_test; + +SELECT +sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) +FROM +sipHashKeyed_test; + +SELECT +sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) +FROM +sipHashKeyed_test; + +SELECT +hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) +FROM +sipHashKeyed_test; + +SELECT +hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) +FROM +sipHashKeyed_test; + +DROP TABLE sipHashKeyed_test; + +SELECT 'Check multiple keys'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); +INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); +SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; +DROP TABLE sipHashKeyed_keys; +SELECT 'Check multiple keys 2'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); +INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); +SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; +SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; +DROP TABLE sipHashKeyed_keys; +SELECT 'Check multiple keys 3'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2); +INSERT INTO sipHashKeyed_keys VALUES (4, 4); +SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; +SELECT '-'; +SELECT sipHash64Keyed((2::UInt64, 2::UInt64), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; +DROP TABLE sipHashKeyed_keys; diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index d00491fd7e5..452e9910660 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -1 +1,130 @@ +A3817F04BA25A8E66DF67214C7550293 +DA87C1D86B99AF44347659119B22FC45 +8177228DA4A45DC7FCA38BDEF60AFFE4 +9C70B60C5267A94E5F33B6B02985ED51 +F88164C12D9C8FAF7D0F6E7C7BCD5579 +1368875980776F8854527A07690E9627 +14EECA338B208613485EA0308FD7A15E +A1F1EBBED8DBC153C0B84AA61FF08239 +3B62A9BA6258F5610F83E264F31497B4 +264499060AD9BAABC47F8B02BB6D71ED +00110DC378146956C95447D3F3D0FBBA +0151C568386B6677A2B4DC6F81E5DC18 +D626B266905EF35882634DF68532C125 +9869E247E9C08B10D029934FC4B952F7 +31FCEFAC66D7DE9C7EC7485FE4494902 +5493E99933B0A8117E08EC0F97CFC3D9 +6EE2A4CA67B054BBFD3315BF85230577 +473D06E8738DB89854C066C47AE47740 +A426E5E423BF4885294DA481FEAEF723 +78017731CF65FAB074D5208952512EB1 +9E25FC833F2290733E9344A5E83839EB +568E495ABE525A218A2214CD3E071D12 +4A29B54552D16B9A469C10528EFF0AAE +C9D184DDD5A9F5E0CF8CE29A9ABF691C +2DB479AE78BD50D8882A8A178A6132AD +8ECE5F042D5E447B5051B9EACB8D8F6F +9C0B53B4B3C307E87EAEE08678141F66 +ABF248AF69A6EAE4BFD3EB2F129EEB94 +0664DA1668574B88B935F3027358AEF4 +AA4B9DC4BF337DE90CD4FD3C467C6AB7 +EA5C7F471FAF6BDE2B1AD7D4686D2287 +2939B0183223FAFC1723DE4F52C43D35 +7C3956CA5EEAFC3E363E9D556546EB68 +77C6077146F01C32B6B69D5F4EA9FFCF +37A6986CB8847EDF0925F0F1309B54DE +A705F0E69DA9A8F907241A2E923C8CC8 +3DC47D1F29C448461E9E76ED904F6711 +0D62BF01E6FC0E1A0D3C4751C5D3692B +8C03468BCA7C669EE4FD5E084BBEE7B5 +528A5BB93BAF2C9C4473CCE5D0D22BD9 +DF6A301E95C95DAD97AE0CC8C6913BD8 +801189902C857F39E73591285E70B6DB +E617346AC9C231BB3650AE34CCCA0C5B +27D93437EFB721AA401821DCEC5ADF89 +89237D9DED9C5E78D8B1C9B166CC7342 +4A6D8091BF5E7D651189FA94A250B14C +0E33F96055E7AE893FFC0E3DCF492902 +E61C432B720B19D18EC8D84BDC63151B +F7E5AEF549F782CF379055A608269B16 +438D030FD0B7A54FA837F2AD201A6403 +A590D3EE4FBF04E3247E0D27F286423F +5FE2C1A172FE93C4B15CD37CAEF9F538 +2C97325CBD06B36EB2133DD08B3A017C +92C814227A6BCA949FF0659F002AD39E +DCE850110BD8328CFBD50841D6911D87 +67F14984C7DA791248E32BB5922583DA +1938F2CF72D54EE97E94166FA91D2A36 +74481E9646ED49FE0F6224301604698E +57FCA5DE98A9D6D8006438D0583D8A1D +9FECDE1CEFDC1CBED4763674D9575359 +E3040C00EB28F15366CA73CBD872E740 +7697009A6A831DFECCA91C5993670F7A +5853542321F567A005D547A4F04759BD +5150D1772F50834A503E069A973FBD7C +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 1 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index 200954c3b57..c238e51b690 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -1 +1,207 @@ +-- Test Vectors from the SipHash reference C implementation: +-- Written by +-- Jean-Philippe Aumasson +-- Daniel J. Bernstein +-- Released under CC0 +-- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645 + +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + '')); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62))); + +-- CH tests +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); + +select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError 48 } +select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError 48 } + SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; +SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; From ed4c560e330194c7a8f9ffd6b676ac6ac256830e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 18 Jul 2023 11:00:13 +0200 Subject: [PATCH 333/473] Accept paths without ancestors --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index e3e89969c17..f23663f9c25 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -357,6 +357,8 @@ void ZooKeeper::createAncestors(const std::string & path) std::vector pending_nodes; size_t last_pos = path.rfind('/'); + if (last_pos == std::string::npos || last_pos == 0) + return; std::string current_node = path.substr(0, last_pos); while (true) From 4f7fd69883fe01c9e6fe5661051cd1bc9ffc49f6 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Mon, 17 Jul 2023 14:58:27 +0300 Subject: [PATCH 334/473] Added function firstLine to extract the first line from a multiline string Fixes #51172 --- .../functions/string-functions.md | 33 +++++++++++++++ .../functions/string-functions.md | 36 ++++++++++++++++ src/Functions/StringHelpers.h | 2 +- src/Functions/firstLine.cpp | 42 +++++++++++++++++++ .../0_stateless/02815_first_line.reference | 9 ++++ .../queries/0_stateless/02815_first_line.sql | 12 ++++++ .../aspell-ignore/en/aspell-dict.txt | 1 + 7 files changed, 134 insertions(+), 1 deletion(-) create mode 100644 src/Functions/firstLine.cpp create mode 100644 tests/queries/0_stateless/02815_first_line.reference create mode 100644 tests/queries/0_stateless/02815_first_line.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 4f174a53ad6..9890d257e84 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1267,3 +1267,36 @@ Like [initcap](#initcap), assuming that the string contains valid UTF-8 encoded Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. + +## firstLine + +Returns the first line from a multi-line string. + +**Syntax** + +```sql +firstLine(val) +``` + +**Arguments** + +- `val` - Input value. [String](../data-types/string.md) + +**Returned value** + +- The first line of the input value or the whole value if there is no line + separators. [String](../data-types/string.md) + +**Example** + +```sql +select firstLine('foo\nbar\nbaz'); +``` + +Result: + +```result +┌─firstLine('foo\nbar\nbaz')─┐ +│ foo │ +└────────────────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index b872200f99b..276dfc2ef20 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -1124,3 +1124,39 @@ Do Nothing for 2 Minutes 2:00   Не учитывает язык. То есть, для турецкого языка, результат может быть не совсем верным. Если длина UTF-8 последовательности байтов различна для верхнего и нижнего регистра кодовой точки, то для этой кодовой точки результат работы может быть некорректным. Если строка содержит набор байтов, не являющийся UTF-8, то поведение не определено. + +## firstLine + +Возвращает первую строку в многострочном тексте. + +**Синтаксис** + +```sql +firstLine(val) +``` + +**Аргументы** + +- `val` - текст для обработки. [String](../data-types/string.md) + +**Returned value** + +- Первая строка текста или весь текст, если переносы строк отсутствуют. + +Тип: [String](../data-types/string.md) + +**Пример** + +Запрос: + +```sql +select firstLine('foo\nbar\nbaz'); +``` + +Результат: + +```result +┌─firstLine('foo\nbar\nbaz')─┐ +│ foo │ +└────────────────────────────┘ +``` diff --git a/src/Functions/StringHelpers.h b/src/Functions/StringHelpers.h index a0f4d18aa80..8f3a87d5d0e 100644 --- a/src/Functions/StringHelpers.h +++ b/src/Functions/StringHelpers.h @@ -156,7 +156,7 @@ struct CutSubstringImpl static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported this function"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by this function"); } }; diff --git a/src/Functions/firstLine.cpp b/src/Functions/firstLine.cpp new file mode 100644 index 00000000000..20b47361d58 --- /dev/null +++ b/src/Functions/firstLine.cpp @@ -0,0 +1,42 @@ +#include +#include +#include +#include + +namespace DB +{ + +struct FirstLine +{ + static size_t getReserveLengthForElement() { return 16; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + + const Pos end = data + size; + const Pos pos = find_first_symbols<'\r', '\n'>(data, end); + res_size = pos - data; + } +}; + +struct NameFirstLine +{ + static constexpr auto name = "firstLine"; +}; + +using FunctionFirstLine = FunctionStringToString, NameFirstLine>; + +REGISTER_FUNCTION(FirstLine) +{ + factory.registerFunction(FunctionDocumentation{ + .description = "Returns first line of a multi-line string.", + .syntax = "firstLine(string)", + .arguments = {{.name = "string", .description = "The string to process."}}, + .returned_value = {"The first line of the string or the whole string if there is no line separators."}, + .examples = { + {.name = "Return first line", .query = "firstLine('Hello\\nWorld')", .result = "'Hello'"}, + {.name = "Return whole string", .query = "firstLine('Hello World')", .result = "'Hello World'"}, + }}); +} +} diff --git a/tests/queries/0_stateless/02815_first_line.reference b/tests/queries/0_stateless/02815_first_line.reference new file mode 100644 index 00000000000..cdc86229cc8 --- /dev/null +++ b/tests/queries/0_stateless/02815_first_line.reference @@ -0,0 +1,9 @@ +foo +foo +foo +foobarbaz +== vector +1 foo +2 quux +3 single line +4 windows diff --git a/tests/queries/0_stateless/02815_first_line.sql b/tests/queries/0_stateless/02815_first_line.sql new file mode 100644 index 00000000000..8c0affaebd3 --- /dev/null +++ b/tests/queries/0_stateless/02815_first_line.sql @@ -0,0 +1,12 @@ +select firstLine('foo\nbar\nbaz'); +select firstLine('foo\rbar\rbaz'); +select firstLine('foo\r\nbar\r\nbaz'); +select firstLine('foobarbaz'); + +select '== vector'; + +drop table if exists 02815_first_line_vector; +create table 02815_first_line_vector (n Int32, text String) engine = MergeTree order by n; + +insert into 02815_first_line_vector values (1, 'foo\nbar\nbaz'), (2, 'quux\n'), (3, 'single line'), (4, 'windows\r\nline breaks'); +select n, firstLine(text) from 02815_first_line_vector order by n; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 6c88d63be49..57934b28728 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1428,6 +1428,7 @@ filesystemFree filesystems finalizeAggregation fips +firstLine firstSignificantSubdomain firstSignificantSubdomainCustom fixedstring From dc55c8ed8eeb4891f8ee7161eb962ee991bacd3d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 18 Jul 2023 09:54:33 +0000 Subject: [PATCH 335/473] Enable no-upgrade-check for 02273_full_sort_join --- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 8b739330364..43f7354017c 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,4 +1,6 @@ --- Tags: long +-- Tags: long, no-upgrade-check + +-- TODO(@vdimir): remove no-upgrade-check tag after https://github.com/ClickHouse/ClickHouse/pull/51737 is released DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 6b00a38aa45d79b72088f608a993579a0fe31d9a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 10:06:44 +0000 Subject: [PATCH 336/473] Remove SystemLogBase (wip) --- src/Common/SystemLogBase.cpp | 96 +++++++++------------------------- src/Common/SystemLogBase.h | 75 ++++++++------------------ src/Interpreters/SystemLog.cpp | 62 +++++++++++++++++----- src/Interpreters/SystemLog.h | 38 +++++++++----- src/Interpreters/TextLog.cpp | 2 +- src/Interpreters/TextLog.h | 4 +- src/Loggers/Loggers.cpp | 5 +- 7 files changed, 127 insertions(+), 155 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index c979613b146..dc6a4a08bb7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -38,46 +38,13 @@ namespace ISystemLog::~ISystemLog() = default; + template -SystemLogBase::SystemLogBase( +SystemLogQueue::SystemLogQueue( const String & name_, - std::shared_ptr> queue_) - : queue(queue_ ? queue_ : std::make_shared>(name_)) -{ -} - -template -void SystemLogBase::stopFlushThread() -{ - { - std::lock_guard lock(queue->mutex); - - if (!saving_thread || !saving_thread->joinable()) - return; - - if (is_shutdown) - return; - - is_shutdown = true; - queue->shutdown(); - - /// Tell thread to shutdown. - queue->flush_event.notify_all(); - } - - saving_thread->join(); -} - -template -void SystemLogBase::startup() -{ - std::lock_guard lock(queue->mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); -} - -template -SystemLogQueue::SystemLogQueue(const String & name_) + size_t flush_interval_milliseconds_) : log(&Poco::Logger::get(name_)) + , flush_interval_milliseconds(flush_interval_milliseconds_) {} static thread_local bool recursive_add_call = false; @@ -149,6 +116,14 @@ void SystemLogQueue::add(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogQueue::shutdown() +{ + is_shutdown = true; + /// Tell thread to shutdown. + flush_event.notify_all(); +} + template uint64_t SystemLogQueue::notifyFlush(bool force) { @@ -193,12 +168,19 @@ void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset } } -constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; +template +void SystemLogQueue::confirm(uint64_t to_flush_end) +{ + std::lock_guard lock(mutex); + flushed_up_to = to_flush_end; + is_force_prepare_tables = false; + flush_event.notify_all(); +} template void SystemLogQueue::pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) { - std::unique_lock lock(queue->mutex); + std::unique_lock lock(mutex); flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds), [&] () @@ -207,47 +189,19 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& } ); - queue_front_index += queue->size(); - to_flush_end = queue->queue_front_index; + queue_front_index += queue.size(); + to_flush_end = queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. output.resize(0); - queue.swap(to_flush); + queue.swap(output); should_prepare_tables_anyway = is_force_prepare_tables; exit_this_thread = is_shutdown; } -template -void SystemLogBase::add(const LogElement & element) -{ - queue->add(element); -} - -template -void SystemLogBase::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - -template -void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } - -// template -// uint64_t SystemLogBase::notifyFlushImpl(bool force) -// { -// return queue->notifyFlush(force); -// } - -#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; +#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) -#define INSTANTIATE_SYSTEM_LOG_BASE2(ELEMENT) template class SystemLogQueue; -SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE2) - } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0c1f32672a9..4cdb07c2cab 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -68,22 +68,33 @@ public: protected: std::unique_ptr saving_thread; + + bool is_shutdown = false; }; template class SystemLogQueue { public: - SystemLogQueue(const String & name_); + SystemLogQueue( + const String & name_, + size_t flush_interval_milliseconds_); + // producer methods void add(const LogElement & element); - size_t size() const { return queue.size(); } - //void push_back(const LogElement & element) { queue.push_back(element); } - void shutdown() { is_shutdown = true; } - + void shutdown(); uint64_t notifyFlush(bool force); void waitFlush(uint64_t this_thread_requested_offset_); - void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) + + // consumer methods + void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread); + void confirm(uint64_t to_flush_end); + + /// Data shared between callers of add()/flush()/shutdown(), and the saving thread + std::mutex mutex; + +private: + Poco::Logger * log; // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; @@ -92,61 +103,19 @@ public: // can wait until a particular message is flushed. This is used to implement // synchronous log flushing for SYSTEM FLUSH LOGS. uint64_t queue_front_index = 0; - - /// Data shared between callers of add()/flush()/shutdown(), and the saving thread - std::mutex mutex; - std::condition_variable flush_event; - - // Requested to flush logs up to this index, exclusive - uint64_t requested_flush_up_to = 0; - // A flag that says we must create the tables even if the queue is empty. bool is_force_prepare_tables = false; - + // Requested to flush logs up to this index, exclusive + uint64_t requested_flush_up_to = 0; + std::condition_variable flush_event; // Flushed log up to this index, exclusive uint64_t flushed_up_to = 0; -private: - Poco::Logger * log; - bool is_shutdown = false; // Logged overflow message at this queue front index uint64_t logged_queue_full_at_index = -1; -}; - -template -class SystemLogBase : public ISystemLog -{ -public: - using Self = SystemLogBase; - - SystemLogBase( - const String & name_, - std::shared_ptr> queue_ = nullptr); - - /** Append a record into log. - * Writing to table will be done asynchronously and in case of failure, record could be lost. - */ - void add(const LogElement & element); - - /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; - - void startup() override; - - void stopFlushThread() override; - - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - - String getName() const override { return LogElement::name(); } - - static const char * getDefaultOrderBy() { return "event_date, event_time"; } - -protected: - Poco::Logger * log; - - std::shared_ptr> queue; bool is_shutdown = false; + + const size_t flush_interval_milliseconds; }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 983c9f87c45..cb4578689a2 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -334,15 +334,21 @@ SystemLog::SystemLog( const String & storage_def_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : Base(database_name_ + "." + table_name_, queue_) - , WithContext(context_) + : WithContext(context_) + , log(&Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")")) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) - , flush_interval_milliseconds(flush_interval_milliseconds_) + , queue(queue_ ? queue_ : std::make_shared>(database_name_ + "." + table_name_, flush_interval_milliseconds_)) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); - log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); +} + +template +void SystemLog::startup() +{ + std::lock_guard lock(queue->mutex); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); } template @@ -355,6 +361,26 @@ void SystemLog::shutdown() table->flushAndShutdown(); } +template +void SystemLog::stopFlushThread() +{ + { + std::lock_guard lock(queue->mutex); + + if (!saving_thread || !saving_thread->joinable()) + return; + + if (is_shutdown) + return; + + is_shutdown = true; + queue->shutdown(); + } + + saving_thread->join(); +} + + template void SystemLog::savingThreadFunction() { @@ -381,9 +407,7 @@ void SystemLog::savingThreadFunction() prepareTable(); LOG_TRACE(log, "Table created (force)"); - std::lock_guard lock(queue->mutex); - queue->is_force_prepare_tables = false; - queue->flush_event.notify_all(); + queue->confirm(to_flush_end); } } else @@ -455,12 +479,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, tryLogCurrentException(__PRETTY_FUNCTION__); } - { - std::lock_guard lock(queue->mutex); - queue->flushed_up_to = to_flush_end; - queue->is_force_prepare_tables = false; - queue->flush_event.notify_all(); - } + queue->confirm(to_flush_end); LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end); } @@ -599,7 +618,24 @@ ASTPtr SystemLog::getCreateTableQuery() return create; } +template +void SystemLog::add(const LogElement & element) +{ + queue->add(element); +} +template +void SystemLog::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLog::notifyFlush(bool force) { queue->notifyFlush(force); } #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index c730aa23319..fe9e4aa35d2 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -89,11 +89,10 @@ struct SystemLogs template -class SystemLog : public SystemLogBase, private boost::noncopyable, WithContext +class SystemLog : public ISystemLog, private boost::noncopyable, WithContext { public: using Self = SystemLog; - using Base = SystemLogBase; /** Parameter: table name where to write log. * If table is not exists, then it get created with specified engine. @@ -111,30 +110,41 @@ public: size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); + void startup() override; + /** Append a record into log. + * Writing to table will be done asynchronously and in case of failure, record could be lost. + */ + void add(const LogElement & element); + void shutdown() override; + String getName() const override { return LogElement::name(); } + static const char * getDefaultOrderBy() { return "event_date, event_time"; } + + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. + void flush(bool force) override; + + /// Non-blocking flush data in the buffer to disk. + void notifyFlush(bool force); + + void stopFlushThread() override; + protected: - //using ISystemLog::mutex; - using Base::is_shutdown; - // using ISystemLog::flush_event; - using Base::stopFlushThread; - using Base::log; - using Base::queue; - // using Base::queue_front_index; - // using Base::is_force_prepare_tables; - //using Base::requested_flush_up_to; - // using Base::flushed_up_to; - // using Base::logged_queue_full_at_index; + Poco::Logger * log; + + using ISystemLog::is_shutdown; + using ISystemLog::saving_thread; private: + /* Saving thread data */ const StorageID table_id; const String storage_def; String create_query; String old_create_query; bool is_prepared = false; - const size_t flush_interval_milliseconds; + std::shared_ptr> queue; /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index a66092c1c2b..108135c78b3 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -84,7 +84,7 @@ TextLog::TextLog(ContextPtr context_, const String & database_name_, const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_) : SystemLog(context_, database_name_, table_name_, - storage_def_, flush_interval_milliseconds_, getLogQueue()) + storage_def_, flush_interval_milliseconds_, getLogQueue(flush_interval_milliseconds_)) { // SystemLog methods may write text logs, so we disable logging for the text // log table to avoid recursion. diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 8390259e147..0febce03abc 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -47,9 +47,9 @@ public: const String & storage_def_, size_t flush_interval_milliseconds_); - static std::shared_ptr> getLogQueue() + static std::shared_ptr> getLogQueue(size_t flush_interval_milliseconds) { - static std::shared_ptr> queue = std::make_shared>("text_log"); + static std::shared_ptr> queue = std::make_shared>("text_log", flush_interval_milliseconds); return queue; } }; diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index fa143440cc2..a9869847f65 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,6 +34,7 @@ static std::string createDirectory(const std::string & file) return path; } +constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { @@ -254,7 +255,9 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log { String text_log_level_str = config.getString("text_log.level", ""); int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); - split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + size_t flush_interval_milliseconds = config.getUInt64("text_log.flush_interval_milliseconds", + DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); + split->addTextLog(DB::TextLog::getLogQueue(flush_interval_milliseconds), text_log_level); } #endif } From e9b7a84aa1c7d4a5ccabed137ed1859ba87ea676 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 12:09:58 +0200 Subject: [PATCH 337/473] Fix network manager for integration tests Sometimes you may get: > raise subprocess.CalledProcessError(exit_code, cmd) E subprocess.CalledProcessError: Command '['iptables', '--wait', '-D', 'DOCKER-USER', '-p', 'tcp', '-s', '172.16.2.3', '-d', '172.16.2.2', '-j', 'DROP']' returned non-zero exit status 137. And only sometimes you may get the reason: OCI runtime exec failed: exec failed: cannot exec in a stopped container: unknown So this means that container for iptables does not exists anymore, and the reason is the timeout. And the fact that container_exit_timeout was equal to container_expire_timeout and was 120. From the docker logs: time="2023-07-16T15:46:52.513673446Z" level=debug msg="form data: {\"AttachStderr\":false,\"AttachStdin\":false,\"AttachStdout\":false,\"Cmd\":[\"sleep\",\"120\"],\"HostConfig\":{\"AutoRemove\":true,\"NetworkMode\":\"host\"},\"Image\":\"clickhouse/integration-helper:latest\",\"NetworkDisabled\":false,\"OpenStdin\":false,\"StdinOnce\":false,\"Tty\":false}" time="2023-07-16T15:48:57.611857183Z" level=debug msg="form data: {\"AttachStderr\":false,\"AttachStdin\":false,\"AttachStdout\":false,\"Cmd\":[\"sleep\",\"120\"],\"HostConfig\":{\"AutoRemove\":true,\"NetworkMode\":\"host\"},\"Image\":\"clickhouse/integration-helper:latest\",\"NetworkDisabled\":false,\"OpenStdin\":false,\"StdinOnce\":false,\"Tty\":false}" And then pytest will try to execute the iptables command: time="2023-07-16T15:50:57.698705244Z" level=debug msg="starting exec command 860920ab2aa07e8d285050f200ac92423a3cf8ec3fb2f57683541e62cf6bc20e in container 66d6c96671b5e987345290ddd260727d96b99789b512d40f333f6263f42fd2f1" Signed-off-by: Azat Khuzhin --- tests/integration/helpers/network.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 60b46926589..e6e79dc7947 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -219,10 +219,15 @@ class _NetworkManager: def __init__( self, - container_expire_timeout=120, - container_exit_timeout=120, + container_expire_timeout=600, + container_exit_timeout=660, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): + # container should be alive for at least 15 seconds then the expiration + # timeout, this is the protection from the case when the container will + # be destroyed just when some test will try to use it. + assert container_exit_timeout >= container_expire_timeout + 15 + self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout From ed60bb90b28e9b325031db6cfcf438fa6a7ca51b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 10:22:07 +0000 Subject: [PATCH 338/473] Move code to be more beautiful in pr --- src/Common/SystemLogBase.cpp | 51 ++++++++++++++++++------------------ 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index dc6a4a08bb7..38d675e4b5f 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -58,7 +58,8 @@ void SystemLogQueue::add(const LogElement & element) return; recursive_add_call = true; SCOPE_EXIT({ recursive_add_call = false; }); - /// Memory can be allocated while resizing on queue.push_back. + + /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. @@ -124,30 +125,6 @@ void SystemLogQueue::shutdown() flush_event.notify_all(); } -template -uint64_t SystemLogQueue::notifyFlush(bool force) -{ - uint64_t this_thread_requested_offset; - - { - std::lock_guard lock(mutex); - if (is_shutdown) - return uint64_t(-1); - - this_thread_requested_offset = queue_front_index + queue.size(); - - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); - - flush_event.notify_all(); - } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; -} - template void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset_) { @@ -201,6 +178,30 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& exit_this_thread = is_shutdown; } +template +uint64_t SystemLogQueue::notifyFlush(bool force) +{ + uint64_t this_thread_requested_offset; + + { + std::lock_guard lock(mutex); + if (is_shutdown) + return uint64_t(-1); + + this_thread_requested_offset = queue_front_index + queue.size(); + + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + is_force_prepare_tables |= force; + requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + + flush_event.notify_all(); + } + + LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); + return this_thread_requested_offset; +} + #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) From 92ca2b0bdd4e2160831d14f1c855a55dbac1a108 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 11:34:28 +0200 Subject: [PATCH 339/473] Make psql tests in test_odbc_interaction more robust to other test failures Signed-off-by: Azat Khuzhin --- .../integration/test_odbc_interaction/test.py | 295 +++++++++--------- 1 file changed, 155 insertions(+), 140 deletions(-) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 14f5de17870..e8b3ba3fcf3 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -582,75 +582,83 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute( - "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" - ) - node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") - node1.exec_in_container( - ["ss", "-K", "dport", "postgresql"], privileged=True, user="root" - ) - node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))", - "hello", - ) - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", - "world", - ) - cursor.execute("truncate table clickhouse.test_table") + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute( + "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" + ) + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") + node1.exec_in_container( + ["ss", "-K", "dport", "postgresql"], privileged=True, user="root" + ) + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))", + "hello", + ) + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", + "world", + ) + finally: + cursor.execute("truncate table clickhouse.test_table") def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')") - for i in range(100): - try: - node1.query("system reload dictionary postgres_odbc_hashed", timeout=15) - except Exception as ex: - assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')") + for i in range(100): + try: + node1.query("system reload dictionary postgres_odbc_hashed", timeout=15) + except Exception as ex: + assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))", - "xxx", - ) - cursor.execute("truncate table clickhouse.test_table") + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))", + "xxx", + ) + finally: + cursor.execute("truncate table clickhouse.test_table") def test_no_connection_pooling(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute( - "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" - ) - node1.exec_in_container(["ss", "-K", "dport", "5432"], privileged=True, user="root") - node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_nopool") - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(1))", - "hello", - ) - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(2))", - "world", - ) + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute( + "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" + ) + node1.exec_in_container( + ["ss", "-K", "dport", "5432"], privileged=True, user="root" + ) + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_nopool") + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(1))", + "hello", + ) + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(2))", + "world", + ) - # No open connections should be left because we don't use connection pooling. - assert "" == node1.exec_in_container( - ["ss", "-H", "dport", "5432"], privileged=True, user="root" - ) - cursor.execute("truncate table clickhouse.test_table") + # No open connections should be left because we don't use connection pooling. + assert "" == node1.exec_in_container( + ["ss", "-H", "dport", "5432"], privileged=True, user="root" + ) + finally: + cursor.execute("truncate table clickhouse.test_table") def test_postgres_insert(started_cluster): @@ -662,112 +670,119 @@ def test_postgres_insert(started_cluster): # postgres .yml file). This is needed to check parsing, validation and # reconstruction of connection string. - node1.query( - "create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" - ) - node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')") - assert node1.query("select * from pg_insert") == "1\t1\thello\n2\t2\tworld\n" - node1.query( - "insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test" - ) - node1.query( - "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" - " select number, number, 's' || toString(number) from numbers (4, 7)" - ) - assert ( - node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n" - ) - assert ( + try: node1.query( - "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))" + "create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" ) - == "55\t10\n" - ) - node1.query("DROP TABLE pg_insert") - conn.cursor().execute("truncate table clickhouse.test_table") + node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')") + assert node1.query("select * from pg_insert") == "1\t1\thello\n2\t2\tworld\n" + node1.query( + "insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test" + ) + node1.query( + "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" + " select number, number, 's' || toString(number) from numbers (4, 7)" + ) + assert ( + node1.query("select sum(column1), count(column1) from pg_insert") + == "55\t10\n" + ) + assert ( + node1.query( + "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))" + ) + == "55\t10\n" + ) + finally: + node1.query("DROP TABLE IF EXISTS pg_insert") + conn.cursor().execute("truncate table clickhouse.test_table") def test_odbc_postgres_date_data_type(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute( - "CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)" - ) + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute( + "CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)" + ) - cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')") - cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')") - cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')") - conn.commit() + cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')") + cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')") + cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')") + conn.commit() - node1.query( - """ - CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date) - ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')""" - ) + node1.query( + """ + CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date) + ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')""" + ) - expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n" - result = node1.query("SELECT * FROM test_date") - assert result == expected - cursor.execute("DROP TABLE clickhouse.test_date") - node1.query("DROP TABLE test_date") + expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n" + result = node1.query("SELECT * FROM test_date") + assert result == expected + finally: + cursor.execute("DROP TABLE clickhouse.test_date") + node1.query("DROP TABLE IF EXISTS test_date") def test_odbc_postgres_conversions(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() - cursor.execute( - """CREATE TABLE clickhouse.test_types ( - a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, - h timestamp)""" - ) + cursor.execute( + """CREATE TABLE clickhouse.test_types ( + a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, + h timestamp)""" + ) - node1.query( - """ - INSERT INTO TABLE FUNCTION - odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') - VALUES (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12')""" - ) + node1.query( + """ + INSERT INTO TABLE FUNCTION + odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') + VALUES (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12')""" + ) - result = node1.query( - """ - SELECT a, b, c, d, e, f, g, h - FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') - """ - ) + result = node1.query( + """ + SELECT a, b, c, d, e, f, g, h + FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') + """ + ) - assert ( - result - == "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\n" - ) - cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") + assert ( + result + == "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\n" + ) + cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") - cursor.execute( - """CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)""" - ) + cursor.execute( + """CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)""" + ) - node1.query( - """ - CREATE TABLE test_types (column1 DateTime64, column2 Decimal(5, 1)) - ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')""" - ) + node1.query( + """ + CREATE TABLE test_types (column1 DateTime64, column2 Decimal(5, 1)) + ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')""" + ) - node1.query( - """INSERT INTO test_types - SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)""" - ) + node1.query( + """INSERT INTO test_types + SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)""" + ) - expected = node1.query( - "SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)" - ) - result = node1.query("SELECT * FROM test_types") - cursor.execute("DROP TABLE clickhouse.test_types") - node1.query("DROP TABLE test_types") - assert result == expected + expected = node1.query( + "SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)" + ) + result = node1.query("SELECT * FROM test_types") + assert result == expected + finally: + cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") + node1.query("DROP TABLE IF EXISTS test_types") def test_odbc_cyrillic_with_varchar(started_cluster): From baf6d182d63dddf9dfb42d687d21ed2835f536a2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 11:49:34 +0200 Subject: [PATCH 340/473] Fix reporting some BROKEN tests as ERROR Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index f2adea78da7..a6e9716dc20 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -788,6 +788,7 @@ class ClickhouseIntegrationTestsRunner: and test not in counters["ERROR"] and test not in counters["SKIPPED"] and test not in counters["FAILED"] + and test not in counters["BROKEN"] and "::" in test ): counters["ERROR"].append(test) From 0cfd12aba46030b2147def929fada009a05eb07c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jul 2023 12:20:56 +0200 Subject: [PATCH 341/473] Fix --- .../MaterializedPostgreSQLConsumer.cpp | 55 +++++++++++++------ .../test.py | 11 ++-- 2 files changed, 45 insertions(+), 21 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index ea7009fc082..ff4ccd97c0d 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -11,6 +11,7 @@ #include #include #include +#include "Common/Exception.h" #include @@ -22,6 +23,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int TOO_MANY_PARTS; } MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( @@ -556,34 +558,49 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl void MaterializedPostgreSQLConsumer::syncTables() { - for (const auto & table_name : tables_to_sync) + while (!tables_to_sync.empty()) { + auto table_name = *tables_to_sync.begin(); auto & storage_data = storages.find(table_name)->second; Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); - if (result_rows.rows()) + try { - auto storage = storage_data.storage; + if (result_rows.rows()) + { + auto storage = storage_data.storage; - auto insert_context = Context::createCopy(context); - insert_context->setInternalQuery(true); + auto insert_context = Context::createCopy(context); + insert_context->setInternalQuery(true); - auto insert = std::make_shared(); - insert->table_id = storage->getStorageID(); - insert->columns = storage_data.buffer.columns_ast; + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + insert->columns = storage_data.buffer.columns_ast; - InterpreterInsertQuery interpreter(insert, insert_context, true); - auto io = interpreter.execute(); - auto input = std::make_shared( - result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); + InterpreterInsertQuery interpreter(insert, insert_context, true); + auto io = interpreter.execute(); + auto input = std::make_shared( + result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); - io.pipeline.complete(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); + io.pipeline.complete(Pipe(std::move(input))); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); + } } + catch (DB::Exception & e) + { + if (e.code() == ErrorCodes::TOO_MANY_PARTS) + { + /// Retry this buffer later. + storage_data.buffer.columns = result_rows.mutateColumns(); + } + throw; + } + + tables_to_sync.erase(tables_to_sync.begin()); } LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); @@ -735,8 +752,12 @@ void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting) /// Read binary changes from replication slot via COPY command (starting from current lsn in a slot). bool MaterializedPostgreSQLConsumer::consume() { - bool slot_empty = true; + if (!tables_to_sync.empty()) + { + syncTables(); + } + bool slot_empty = true; try { auto tx = std::make_shared(connection->getRef()); diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2b17024f417..51e980f0423 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -545,7 +545,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", + database_name=clickhouse_postgres_db, + schema_name=schema_name, + postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" @@ -695,15 +697,16 @@ def test_too_many_parts(started_cluster): time.sleep(1) print(f"wait sync try {i}") if instance2.contains_in_log("DB::Exception: Too many parts"): - num = num - 1 break assert num == int( instance2.query("SELECT count() FROM test_database.test_table") - ) + ) or num - 1 == int(instance2.query("SELECT count() FROM test_database.test_table")) assert instance2.contains_in_log("DB::Exception: Too many parts") print(num) - assert num == int(instance2.query("SELECT count() FROM test_database.test_table")) + assert num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) or num - 1 == int(instance2.query("SELECT count() FROM test_database.test_table")) instance2.query("SYSTEM START MERGES") check_tables_are_synchronized( From 722b3859083daf255f7dcde1d188cb9521d37962 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 18 Jul 2023 12:26:42 +0200 Subject: [PATCH 342/473] List replication queue only for current test database --- .../0_stateless/00834_kill_mutation_replicated_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 6797e9cac12..2e917f67fe8 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -36,7 +36,7 @@ ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r2" # Should be empty, but in case of problems we will see some diagnostics -${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.replication_queue WHERE table like 'kill_mutation_r%'" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.replication_queue WHERE database = '$CLICKHOUSE_DATABASE' AND table like 'kill_mutation_r%'" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation_r1 DELETE WHERE toUInt32(s) = 1" From f3bfc6e5cf8476734110e9a0037c82120ca64eb4 Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 11:40:57 +0100 Subject: [PATCH 343/473] Update with.md I propose changes to make it clear that CTEs do not guarantee the same results in all places they are used, since the query will be re-executed in each place they are used. --- docs/en/sql-reference/statements/select/with.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 4654f249548..f841d497dbb 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -5,7 +5,9 @@ sidebar_label: WITH # WITH Clause -ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)), that is provides to use results of `WITH` clause in the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. +ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. + +Note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. ## Syntax From a74a56b0d7026d187783da861ef8eb4840947764 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 11:06:16 +0000 Subject: [PATCH 344/473] Attempt to fix assert in tsan with fibers --- src/Common/AsyncTaskExecutor.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncTaskExecutor.cpp b/src/Common/AsyncTaskExecutor.cpp index b824a0a5b31..923dad1d346 100644 --- a/src/Common/AsyncTaskExecutor.cpp +++ b/src/Common/AsyncTaskExecutor.cpp @@ -5,7 +5,6 @@ namespace DB AsyncTaskExecutor::AsyncTaskExecutor(std::unique_ptr task_) : task(std::move(task_)) { - createFiber(); } void AsyncTaskExecutor::resume() @@ -13,6 +12,10 @@ void AsyncTaskExecutor::resume() if (routine_is_finished) return; + /// Create fiber lazily on first resume() call. + if (!fiber) + createFiber(); + if (!checkBeforeTaskResume()) return; @@ -22,6 +25,11 @@ void AsyncTaskExecutor::resume() return; resumeUnlocked(); + + /// Destroy fiber when it's finished. + if (routine_is_finished) + destroyFiber(); + if (exception) processException(exception); } @@ -46,9 +54,8 @@ void AsyncTaskExecutor::cancel() void AsyncTaskExecutor::restart() { std::lock_guard guard(fiber_lock); - if (fiber) + if (!routine_is_finished) destroyFiber(); - createFiber(); routine_is_finished = false; } From f163953b277b6ab1bc3d0aa0c68d8f7a65ce28c0 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 13:52:59 +0200 Subject: [PATCH 345/473] Use error name instead of number in tests --- tests/queries/0_stateless/02534_keyed_siphash.sql | 8 ++++---- tests/queries/0_stateless/02552_siphash128_reference.sql | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 70ec8470ca3..e780e1df167 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -263,10 +263,10 @@ select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash64Keyed((0, 0), '1'); -- { serverError 48 } -select sipHash128Keyed((0, 0), '1'); -- { serverError 48 } -select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 } -select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 } +select sipHash64Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash64Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index c238e51b690..8b27ecbe61c 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -200,8 +200,8 @@ select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError 48 } -select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError 48 } +select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; From 00d4a0790905360eae940d08becf01176e272ef1 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 14:02:48 +0200 Subject: [PATCH 346/473] Improve tests formatting --- .../0_stateless/02534_keyed_siphash.reference | 10 ++-- .../0_stateless/02534_keyed_siphash.sql | 54 +++++-------------- 2 files changed, 17 insertions(+), 47 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index ea77e5c3963..b6dd2a1c9a3 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -209,17 +209,15 @@ Check const columns 15080046610211022027 2E779C73D13981AA1AE19AFF9617EA49 2E779C73D13981AA1AE19AFF9617EA49 -Check multiple keys +Check multiple keys as tuple from a table 11862823756610506724 9357996107237883963 -Check multiple keys 2 +Check multiple keys as separate ints from a table 11862823756610506724 9357996107237883963 +Check constant key and data from a table 11862823756610506724 11862823756610506724 -Check multiple keys 3 +Check multiple keys as separate ints from a table with constant data 11862823756610506724 9357996107237883963 -- -11862823756610506724 -11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index e780e1df167..5c6789bc305 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -290,65 +290,37 @@ SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt6 SELECT 'Check const columns'; DROP TABLE IF EXISTS sipHashKeyed_test; - -CREATE TABLE sipHashKeyed_test -ENGINE = Memory() -AS -SELECT -1 a, -'test' b; - -SELECT -sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); - -SELECT -sipHash64(tuple(*)) -FROM -sipHashKeyed_test; - -SELECT -sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) -FROM -sipHashKeyed_test; - -SELECT -sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) -FROM -sipHashKeyed_test; - -SELECT -hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) -FROM -sipHashKeyed_test; - -SELECT -hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) -FROM -sipHashKeyed_test; - +CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; +SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); +SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test; +SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test; +SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test; +SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; +SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; DROP TABLE sipHashKeyed_test; -SELECT 'Check multiple keys'; +SELECT 'Check multiple keys as tuple from a table'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; DROP TABLE sipHashKeyed_keys; -SELECT 'Check multiple keys 2'; + +SELECT 'Check multiple keys as separate ints from a table'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; +SELECT 'Check constant key and data from a table'; SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; DROP TABLE sipHashKeyed_keys; -SELECT 'Check multiple keys 3'; + +SELECT 'Check multiple keys as separate ints from a table with constant data'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES (2, 2); INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; -SELECT '-'; -SELECT sipHash64Keyed((2::UInt64, 2::UInt64), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; From 5a398f9a9324be4bdc126f782c01abbe4a9f5b29 Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 13:39:53 +0100 Subject: [PATCH 347/473] Update with.md Added an example --- .../en/sql-reference/statements/select/with.md | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index f841d497dbb..3a74378008a 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -7,7 +7,23 @@ sidebar_label: WITH ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. -Note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. +Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.
+An example of such behavior is below +``` sql +with cte_numbers as +( + select + num + from generateRandom('num UInt64', NULL) + limit 1000000 +) +select + count() +from cte_numbers +where num in (select num from cte_numbers) +``` +If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`
+However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on... ## Syntax From 7466ef58376810d7a81a1e48ac14937d336cd97f Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 13:05:20 +0000 Subject: [PATCH 348/473] Add index --- src/Common/SystemLogBase.cpp | 70 ++++++++++++++++----------------- src/Common/SystemLogBase.h | 27 +++++++------ src/Interpreters/SystemLog.cpp | 4 +- src/Loggers/OwnSplitChannel.cpp | 2 +- 4 files changed, 53 insertions(+), 50 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 38d675e4b5f..e9442617acd 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -47,17 +47,17 @@ SystemLogQueue::SystemLogQueue( , flush_interval_milliseconds(flush_interval_milliseconds_) {} -static thread_local bool recursive_add_call = false; +static thread_local bool recursive_push_call = false; template -void SystemLogQueue::add(const LogElement & element) +void SystemLogQueue::push(const LogElement & element) { /// It is possible that the method will be called recursively. /// Better to drop these events to avoid complications. - if (recursive_add_call) + if (recursive_push_call) return; - recursive_add_call = true; - SCOPE_EXIT({ recursive_add_call = false; }); + recursive_push_call = true; + SCOPE_EXIT({ recursive_push_call = false; }); /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. @@ -118,15 +118,31 @@ void SystemLogQueue::add(const LogElement & element) } template -void SystemLogQueue::shutdown() -{ - is_shutdown = true; - /// Tell thread to shutdown. - flush_event.notify_all(); +uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) +{ + uint64_t this_thread_requested_offset; + + { + std::lock_guard lock(mutex); + if (is_shutdown) + return uint64_t(-1); + + this_thread_requested_offset = queue_front_index + queue.size(); + + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + is_force_prepare_tables |= should_prepare_tables_anyway; + requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + + flush_event.notify_all(); + } + + LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); + return this_thread_requested_offset; } template -void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset_) +void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) { // Use an arbitrary timeout to avoid endless waiting. 60s proved to be // too fast for our parallel functional tests, probably because they @@ -135,7 +151,7 @@ void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset std::unique_lock lock(mutex); bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { - return flushed_up_to >= this_thread_requested_offset_ && !is_force_prepare_tables; + return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; }); if (!result) @@ -155,7 +171,7 @@ void SystemLogQueue::confirm(uint64_t to_flush_end) } template -void SystemLogQueue::pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) +SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) { std::unique_lock lock(mutex); flush_event.wait_for(lock, @@ -167,7 +183,6 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& ); queue_front_index += queue.size(); - to_flush_end = queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. output.resize(0); @@ -176,30 +191,15 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& should_prepare_tables_anyway = is_force_prepare_tables; exit_this_thread = is_shutdown; + return queue_front_index; } template -uint64_t SystemLogQueue::notifyFlush(bool force) -{ - uint64_t this_thread_requested_offset; - - { - std::lock_guard lock(mutex); - if (is_shutdown) - return uint64_t(-1); - - this_thread_requested_offset = queue_front_index + queue.size(); - - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); - - flush_event.notify_all(); - } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; +void SystemLogQueue::shutdown() +{ + is_shutdown = true; + /// Tell thread to shutdown. + flush_event.notify_all(); } #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 4cdb07c2cab..281cd06354a 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -75,20 +75,23 @@ protected: template class SystemLogQueue { + using Index = uint64_t; + public: SystemLogQueue( const String & name_, size_t flush_interval_milliseconds_); - // producer methods - void add(const LogElement & element); void shutdown(); - uint64_t notifyFlush(bool force); - void waitFlush(uint64_t this_thread_requested_offset_); + + // producer methods + void push(const LogElement & element); + Index notifyFlush(bool should_prepare_tables_anyway); + void waitFlush(Index expected_flushed_up_to); // consumer methods - void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread); - void confirm(uint64_t to_flush_end); + Index pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread); + void confirm(Index to_flush_end); /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; @@ -102,19 +105,19 @@ private: // We use it to give a global sequential index to every message, so that we // can wait until a particular message is flushed. This is used to implement // synchronous log flushing for SYSTEM FLUSH LOGS. - uint64_t queue_front_index = 0; + Index queue_front_index = 0; // A flag that says we must create the tables even if the queue is empty. bool is_force_prepare_tables = false; // Requested to flush logs up to this index, exclusive - uint64_t requested_flush_up_to = 0; - std::condition_variable flush_event; + Index requested_flush_up_to = 0; // Flushed log up to this index, exclusive - uint64_t flushed_up_to = 0; + Index flushed_up_to = 0; // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; - + Index logged_queue_full_at_index = -1; + bool is_shutdown = false; + std::condition_variable flush_event; const size_t flush_interval_milliseconds; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index cb4578689a2..3193baa551f 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -398,7 +398,7 @@ void SystemLog::savingThreadFunction() // Should we prepare table even if there are no new messages. bool should_prepare_tables_anyway = false; - queue->pop(to_flush, to_flush_end, should_prepare_tables_anyway, exit_this_thread); + to_flush_end = queue->pop(to_flush, should_prepare_tables_anyway, exit_this_thread); if (to_flush.empty()) { @@ -621,7 +621,7 @@ ASTPtr SystemLog::getCreateTableQuery() template void SystemLog::add(const LogElement & element) { - queue->add(element); + queue->push(element); } template diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 17806153905..b5ac42d6041 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -138,7 +138,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) std::shared_ptr> text_log_locked{}; text_log_locked = text_log.lock(); if (text_log_locked) - text_log_locked->add(elem); + text_log_locked->push(elem); } #endif } From 681bf10c9319722dd32eaf11abd2f42b07452ab4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jul 2023 13:06:02 +0000 Subject: [PATCH 349/473] Check nullptr in fuzzer --- src/Client/QueryFuzzer.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 39ca7a5ed88..5ce95c82528 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -848,6 +848,9 @@ ASTs QueryFuzzer::getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query) void QueryFuzzer::notifyQueryFailed(ASTPtr ast) { + if (ast == nullptr) + return; + auto remove_fuzzed_table = [this](const auto & table_name) { auto pos = table_name.find("__fuzz_"); From cfcfb83ab739a9271a5da8618ee372005f76c357 Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 14:41:43 +0100 Subject: [PATCH 350/473] Update with.md replaced
to \ --- docs/en/sql-reference/statements/select/with.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 3a74378008a..662262efb2b 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -7,7 +7,7 @@ sidebar_label: WITH ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. -Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.
+Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.\ An example of such behavior is below ``` sql with cte_numbers as @@ -22,7 +22,7 @@ select from cte_numbers where num in (select num from cte_numbers) ``` -If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`
+If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`\ However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on... ## Syntax From 9b7ecbaa277bc29962cccd1e0faf379fbbef36af Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Tue, 18 Jul 2023 22:00:47 +0800 Subject: [PATCH 351/473] fix issue#50582 The buildInputOrderInfo() method in optimizeReadInOrder.cpp adds constant columns to order_key_prefix_descr. However, since header_without_constants does not contain constant columns, the constant columns in description_sorted_ also need to be removed in advance. --- .../Transforms/FinishSortingTransform.cpp | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 05fddc35e15..dd61472bc37 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -35,9 +35,25 @@ FinishSortingTransform::FinishSortingTransform( "Can't finish sorting. SortDescription " "of already sorted stream is not prefix of SortDescription needed to sort"); + /// Remove constants from description_sorted_. + SortDescription description_sorted_without_constants; + description_sorted_without_constants.reserve(description_sorted_.size()); + size_t num_columns = header.columns(); + ColumnNumbers map(num_columns, num_columns); + for (const auto & column_description : description_sorted_) + { + auto old_pos = header.getPositionByName(column_description.column_name); + auto new_pos = map[old_pos]; + + if (new_pos < num_columns) + { + description_sorted_without_constants.push_back(column_description); + } + } + /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. - for (const auto & column_sort_desc : description_sorted_) + for (const auto & column_sort_desc : description_sorted_without_constants) description_with_positions.emplace_back(column_sort_desc, header_without_constants.getPositionByName(column_sort_desc.column_name)); } From 90df922f502289e17f29c6314cd0be578741335a Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 15:02:59 +0100 Subject: [PATCH 352/473] Update with.md Removed new line chars completely just in case --- docs/en/sql-reference/statements/select/with.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 662262efb2b..a59ef463419 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -7,7 +7,8 @@ sidebar_label: WITH ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. -Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.\ +Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. + An example of such behavior is below ``` sql with cte_numbers as @@ -22,7 +23,8 @@ select from cte_numbers where num in (select num from cte_numbers) ``` -If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`\ +If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000` + However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on... ## Syntax From 5282fc2c3450535b2fe2854e4cb1176faed3cba3 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:04:17 +0200 Subject: [PATCH 353/473] Fix class name --- src/Functions/FunctionsHashing.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 20612e91a62..ef63f5827fc 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -423,7 +423,7 @@ struct SipHash128ReferenceImpl using ReturnType = UInt128; - static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); } @@ -1853,10 +1853,10 @@ using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; -using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; using FunctionSipHash128ReferenceKeyed - = FunctionAnyHash; + = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; From be0096e536140f62c7ac9600fc6f637a5638ace5 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:05:13 +0200 Subject: [PATCH 354/473] Add more sipHash128 tests --- .../0_stateless/02534_keyed_siphash.reference | 13 ++++++++++++- tests/queries/0_stateless/02534_keyed_siphash.sql | 9 ++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index b6dd2a1c9a3..de783d7dddf 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -197,8 +197,9 @@ E28DBDE7FE22E41C Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 +86AE90BB6A238D3F6221457630142C9B +86AE90BB6A238D3F6221457630142C9B Check memsan bug -1CE422FEE7BD8DE20000000000000000 18096612095653370192 20AF99D3A87829E0 12489502208762728797 @@ -209,15 +210,25 @@ Check const columns 15080046610211022027 2E779C73D13981AA1AE19AFF9617EA49 2E779C73D13981AA1AE19AFF9617EA49 +2E779C73D13981AA1AE19AFF9617EA49 +2E779C73D13981AA1AE19AFF9617EA49 Check multiple keys as tuple from a table 11862823756610506724 9357996107237883963 +86AE90BB6A238D3F6221457630142C9B +F6D93D8FEA6D7DECCDD95A7A0A2AA36D Check multiple keys as separate ints from a table 11862823756610506724 9357996107237883963 +86AE90BB6A238D3F6221457630142C9B +F6D93D8FEA6D7DECCDD95A7A0A2AA36D Check constant key and data from a table 11862823756610506724 11862823756610506724 +86AE90BB6A238D3F6221457630142C9B +86AE90BB6A238D3F6221457630142C9B Check multiple keys as separate ints from a table with constant data 11862823756610506724 9357996107237883963 +86AE90BB6A238D3F6221457630142C9B +F6D93D8FEA6D7DECCDD95A7A0A2AA36D diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 5c6789bc305..14b422ac713 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -280,10 +280,11 @@ INSERT INTO tab VALUES ((2, 2), 4); -- these two statements must produce the same result SELECT sipHash64Keyed(key, val) FROM tab; SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; +SELECT hex(sipHash128Keyed(key, val)) FROM tab; +SELECT hex(sipHash128Keyed(key, 4::UInt64)) FROM tab; DROP TABLE tab; SELECT 'Check memsan bug'; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); @@ -295,6 +296,8 @@ SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test; SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test; SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test; +SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), 1, 'test')); +SELECT hex(sipHash128(tuple(*))) FROM sipHashKeyed_test; SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; DROP TABLE sipHashKeyed_test; @@ -305,6 +308,7 @@ CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Me INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; +SELECT hex(sipHash128Keyed(key, val)) FROM sipHashKeyed_keys ORDER by key; DROP TABLE sipHashKeyed_keys; SELECT 'Check multiple keys as separate ints from a table'; @@ -313,8 +317,10 @@ CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Mem INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; +SELECT hex(sipHash128Keyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; SELECT 'Check constant key and data from a table'; SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; +SELECT hex(sipHash128Keyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; DROP TABLE sipHashKeyed_keys; SELECT 'Check multiple keys as separate ints from a table with constant data'; @@ -323,4 +329,5 @@ CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES (2, 2); INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; +SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; From 388d090e538dd6e5f229f2c036bece7720097f69 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:05:53 +0200 Subject: [PATCH 355/473] Add more sipHash128Reference tests --- .../02552_siphash128_reference.reference | 22 +++++++++ .../02552_siphash128_reference.sql | 47 +++++++++++++++++++ 2 files changed, 69 insertions(+) diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index 452e9910660..ece9f6a4615 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -128,3 +128,25 @@ E3040C00EB28F15366CA73CBD872E740 1 1 1 +Check bug with hashing of const integer values +E940B12600C844966162FF8FE7A16AAE +E940B12600C844966162FF8FE7A16AAE +Check memsan bug +1CE422FEE7BD8DE20000000000000000 +Check const columns +B66B53476BDBEB8549A257E3B1766C30 +B66B53476BDBEB8549A257E3B1766C30 +B66B53476BDBEB8549A257E3B1766C30 +B66B53476BDBEB8549A257E3B1766C30 +Check multiple keys as tuple from a table +E940B12600C844966162FF8FE7A16AAE +EC58946A98A0D37F4E3FAC02FBBA9480 +Check multiple keys as separate ints from a table +E940B12600C844966162FF8FE7A16AAE +EC58946A98A0D37F4E3FAC02FBBA9480 +Check constant key and data from a table +E940B12600C844966162FF8FE7A16AAE +E940B12600C844966162FF8FE7A16AAE +Check multiple keys as separate ints from a table with constant data +E940B12600C844966162FF8FE7A16AAE +EC58946A98A0D37F4E3FAC02FBBA9480 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index 8b27ecbe61c..f7324ed0ee4 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -205,3 +205,50 @@ select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMEN SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +SELECT 'Check bug with hashing of const integer values'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO tab VALUES ((2, 2), 4); +-- these two statements must produce the same result +SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM tab; +SELECT hex(sipHash128ReferenceKeyed(key, 4::UInt64)) FROM tab; +DROP TABLE tab; + +SELECT 'Check memsan bug'; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); + +SELECT 'Check const columns'; +DROP TABLE IF EXISTS sipHashKeyed_test; +CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), 1, 'test')); +SELECT hex(sipHash128Reference(tuple(*))) FROM sipHashKeyed_test; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; +DROP TABLE sipHashKeyed_test; + +SELECT 'Check multiple keys as tuple from a table'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); +INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); +SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM sipHashKeyed_keys ORDER by key; +DROP TABLE sipHashKeyed_keys; + +SELECT 'Check multiple keys as separate ints from a table'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); +INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); +SELECT hex(sipHash128ReferenceKeyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; +SELECT 'Check constant key and data from a table'; +SELECT hex(sipHash128ReferenceKeyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; +DROP TABLE sipHashKeyed_keys; + +SELECT 'Check multiple keys as separate ints from a table with constant data'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2); +INSERT INTO sipHashKeyed_keys VALUES (4, 4); +SELECT hex(sipHash128ReferenceKeyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; +DROP TABLE sipHashKeyed_keys; From f6bbc4bce926f378ec07655022e897182bdcd4c4 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:37:10 +0200 Subject: [PATCH 356/473] Use getData instead of get64 --- src/Functions/FunctionsHashing.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index ef63f5827fc..4965d1f7b49 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -95,7 +95,9 @@ namespace impl { if (is_const) i = 0; - return {key0->get64(i), key1->get64(i)}; + const auto & key0data = assert_cast(*key0).getData(); + const auto & key1data = assert_cast(*key1).getData(); + return {key0data[i], key1data[i]}; } }; From a06631f0dbaa7402966d69c8b4cac69761627b2d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 18 Jul 2023 16:26:10 +0200 Subject: [PATCH 357/473] Better detecting invalid figures --- src/Functions/geoToS2.cpp | 7 +++++++ src/Functions/s2RectAdd.cpp | 13 +++++++++---- src/Functions/s2RectContains.cpp | 13 +++++++++---- src/Functions/s2RectIntersection.cpp | 12 ++++++------ src/Functions/s2RectUnion.cpp | 12 ++++++------ src/Functions/s2ToGeo.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 2 +- .../0_stateless/02816_s2_invalid_point.reference | 0 .../queries/0_stateless/02816_s2_invalid_point.sql | 1 + 9 files changed, 40 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02816_s2_invalid_point.reference create mode 100644 tests/queries/0_stateless/02816_s2_invalid_point.sql diff --git a/src/Functions/geoToS2.cpp b/src/Functions/geoToS2.cpp index c1f333f63e6..8d065b01c34 100644 --- a/src/Functions/geoToS2.cpp +++ b/src/Functions/geoToS2.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; } namespace @@ -108,6 +109,12 @@ public: /// S2 acceptes point as (latitude, longitude) S2LatLng lat_lng = S2LatLng::FromDegrees(lat, lon); + + if (!lat_lng.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive."); + S2CellId id(lat_lng); dst_data[row] = id.id(); diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index e086fdd6b3a..9266f4ae1a7 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -114,13 +114,18 @@ public: const auto hi = S2CellId(data_hi[row]); const auto point = S2CellId(data_point[row]); - if (!lo.is_valid() || !hi.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Rectangle is not valid"); + S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); if (!point.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive."); - S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); + if (!rect.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); rect.AddPoint(point.ToPoint()); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index e4d74ee2545..aed123ce8ee 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -107,13 +107,18 @@ public: const auto hi = S2CellId(data_hi[row]); const auto point = S2CellId(data_point[row]); - if (!lo.is_valid() || !hi.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Rectangle is not valid"); + S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); if (!point.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive."); - S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); + if (!rect.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); dst_data.emplace_back(rect.Contains(point.ToLatLng())); } diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index 072c7147809..ffe26d171d0 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -128,15 +128,15 @@ public: const auto lo2 = S2CellId(data_lo2[row]); const auto hi2 = S2CellId(data_hi2[row]); - if (!lo1.is_valid() || !hi1.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "First rectangle is not valid"); - - if (!lo2.is_valid() || !hi2.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second rectangle is not valid"); - S2LatLngRect rect1(lo1.ToLatLng(), hi1.ToLatLng()); S2LatLngRect rect2(lo2.ToLatLng(), hi2.ToLatLng()); + if (!rect1.is_valid() || !rect2.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); + S2LatLngRect rect_intersection = rect1.Intersection(rect2); vec_res_first.emplace_back(S2CellId(rect_intersection.lo()).id()); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index bb63229b484..472b30c2d55 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -126,15 +126,15 @@ public: const auto lo2 = S2CellId(data_lo2[row]); const auto hi2 = S2CellId(data_hi2[row]); - if (!lo1.is_valid() || !hi1.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "First rectangle is not valid"); - - if (!lo2.is_valid() || !hi2.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second rectangle is not valid"); - S2LatLngRect rect1(lo1.ToLatLng(), hi1.ToLatLng()); S2LatLngRect rect2(lo2.ToLatLng(), hi2.ToLatLng()); + if (!rect1.is_valid() || !rect2.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); + S2LatLngRect rect_union = rect1.Union(rect2); vec_res_first.emplace_back(S2CellId(rect_union.lo()).id()); diff --git a/src/Functions/s2ToGeo.cpp b/src/Functions/s2ToGeo.cpp index 5c1dbfa0382..61cc8212b39 100644 --- a/src/Functions/s2ToGeo.cpp +++ b/src/Functions/s2ToGeo.cpp @@ -97,7 +97,7 @@ public: const auto id = S2CellId(data_id[row]); if (!id.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "CellId is invalid."); S2Point point = id.ToPoint(); S2LatLng ll(point); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index bfaa00eac2c..6a68d0f21f7 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -337,7 +337,7 @@ public: { { std::lock_guard lock(mutex); - queue.emplace(file_segment->key(), file_segment->offset(), file_segment); + queue.push(DownloadInfo{file_segment->key(), file_segment->offset(), file_segment}); } CurrentMetrics::add(CurrentMetrics::FilesystemCacheDownloadQueueElements); diff --git a/tests/queries/0_stateless/02816_s2_invalid_point.reference b/tests/queries/0_stateless/02816_s2_invalid_point.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02816_s2_invalid_point.sql b/tests/queries/0_stateless/02816_s2_invalid_point.sql new file mode 100644 index 00000000000..6f7620d98da --- /dev/null +++ b/tests/queries/0_stateless/02816_s2_invalid_point.sql @@ -0,0 +1 @@ +SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } From f091baaa0faf7fbd63460c486a4853bb66b34b5b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 14:33:26 +0000 Subject: [PATCH 358/473] Remove whitespaces --- src/Common/SystemLogBase.cpp | 4 ++-- src/Common/SystemLogBase.h | 2 +- src/Interpreters/SystemLog.h | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index e9442617acd..1d0673e30dd 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -196,8 +196,8 @@ SystemLogQueue::Index SystemLogQueue::pop(std::vector void SystemLogQueue::shutdown() -{ - is_shutdown = true; +{ + is_shutdown = true; /// Tell thread to shutdown. flush_event.notify_all(); } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 281cd06354a..9436137d4a8 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -114,7 +114,7 @@ private: Index flushed_up_to = 0; // Logged overflow message at this queue front index Index logged_queue_full_at_index = -1; - + bool is_shutdown = false; std::condition_variable flush_event; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index fe9e4aa35d2..6f61e075b49 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -128,10 +128,10 @@ public: void notifyFlush(bool force); void stopFlushThread() override; - + protected: Poco::Logger * log; - + using ISystemLog::is_shutdown; using ISystemLog::saving_thread; From 680903e89bb6923f5b2fd49f6111d8506f5e5c16 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jul 2023 16:54:33 +0200 Subject: [PATCH 359/473] Fix style check --- .../test_postgresql_replica_database_engine_2/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index a6b2b7c498c..3f2ec74180b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -700,7 +700,9 @@ def test_too_many_parts(started_cluster): break assert num == int( instance2.query("SELECT count() FROM test_database.test_table") - ) or num - 1 == int(instance2.query("SELECT count() FROM test_database.test_table")) + ) or num - 1 == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) assert instance2.contains_in_log("DB::Exception: Too many parts") print(num) From 97b2893f78257b0bee60c688666c343be3de0620 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 18:03:56 +0300 Subject: [PATCH 360/473] Update 02816_s2_invalid_point.sql --- tests/queries/0_stateless/02816_s2_invalid_point.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02816_s2_invalid_point.sql b/tests/queries/0_stateless/02816_s2_invalid_point.sql index 6f7620d98da..590eb8b5ec2 100644 --- a/tests/queries/0_stateless/02816_s2_invalid_point.sql +++ b/tests/queries/0_stateless/02816_s2_invalid_point.sql @@ -1 +1,3 @@ +-- Tags: no-fasttest + SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } From 25eee81746609cead61f6cf929bc595102893915 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 18 Jul 2023 16:29:21 +0000 Subject: [PATCH 361/473] Fix temp data deletion on startup, add test --- src/Disks/TemporaryFileOnDisk.cpp | 4 -- src/Disks/TemporaryFileOnDisk.h | 3 +- .../test_temporary_data/__init__.py | 0 tests/integration/test_temporary_data/test.py | 57 +++++++++++++++++++ 4 files changed, 58 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_temporary_data/__init__.py create mode 100644 tests/integration/test_temporary_data/test.py diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 6fe6fd5a1c9..8e5c8bcebbd 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -23,10 +23,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_) - : TemporaryFileOnDisk(disk_, "") -{} - TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Metric metric_scope) : TemporaryFileOnDisk(disk_) { diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index 4c376383087..bd82b9744ea 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -16,9 +16,8 @@ using DiskPtr = std::shared_ptr; class TemporaryFileOnDisk { public: - explicit TemporaryFileOnDisk(const DiskPtr & disk_); explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Metric metric_scope); - explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix); + explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix = "tmp"); ~TemporaryFileOnDisk(); diff --git a/tests/integration/test_temporary_data/__init__.py b/tests/integration/test_temporary_data/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_temporary_data/test.py b/tests/integration/test_temporary_data/test.py new file mode 100644 index 00000000000..9228da0698f --- /dev/null +++ b/tests/integration/test_temporary_data/test.py @@ -0,0 +1,57 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_tmp_data_no_leftovers(start_cluster): + q = node.get_query_request + + settings = { + "max_bytes_before_external_group_by": "10K", + "max_bytes_before_external_sort": "10K", + "join_algorithm": "grace_hash", + "max_bytes_in_join": "10K", + "grace_hash_join_initial_buckets": "16", + } + + # Run some queries in the background to generate temporary data + q( + "SELECT ignore(*) FROM numbers(10 * 1024 * 1024) ORDER BY sipHash64(number)", + settings=settings, + ) + q("SELECT * FROM system.numbers GROUP BY ALL", settings=settings) + q( + "SELECT * FROM system.numbers as t1 JOIN system.numbers as t2 USING (number)", + settings=settings, + ) + + # Wait a bit to make sure the temporary data is written to disk + time.sleep(5) + + # Hard restart the node + node.restart_clickhouse(kill=True) + path_to_data = "/var/lib/clickhouse/" + + # Check that there are no temporary files left + result = node.exec_in_container(["ls", path_to_data + "tmp/"]) + assert result == "" From 89552bbde5f7f452bb07c6b5c9fb1da324524620 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jul 2023 18:39:20 +0200 Subject: [PATCH 362/473] Pull logs for first replica before optimize (#52032) Co-authored-by: Alexander Tokmakov --- .../00502_custom_partitioning_replicated_zookeeper_long.sql | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 0ee8ba07006..a5e33bffb0d 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -15,6 +15,7 @@ INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica1_00502' AND active ORDER BY name; +SYSTEM SYNC REPLICA not_partitioned_replica1_00502 PULL; SYSTEM SYNC REPLICA not_partitioned_replica2_00502; OPTIMIZE TABLE not_partitioned_replica1_00502 PARTITION tuple() FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -42,6 +43,7 @@ INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03' SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica1' AND active ORDER BY name; +SYSTEM SYNC REPLICA partitioned_by_week_replica1 PULL; SYSTEM SYNC REPLICA partitioned_by_week_replica2; OPTIMIZE TABLE partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL; SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync. @@ -68,6 +70,7 @@ INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2 SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple_replica1_00502' AND active ORDER BY name; +SYSTEM SYNC REPLICA partitioned_by_tuple_replica1_00502 PULL; SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00502; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-01', 1) FINAL; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-02', 1) FINAL; @@ -95,6 +98,7 @@ INSERT INTO partitioned_by_string_replica1 VALUES ('bbb', 4), ('aaa', 5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_string_replica1' AND active ORDER BY name; +SYSTEM SYNC REPLICA partitioned_by_string_replica1 PULL; SYSTEM SYNC REPLICA partitioned_by_string_replica2; OPTIMIZE TABLE partitioned_by_string_replica2 PARTITION 'aaa' FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -119,6 +123,7 @@ CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMerg INSERT INTO without_fixed_size_columns_replica1 VALUES ('a'), ('aa'), ('b'), ('cc'); -- Wait for replication. +SYSTEM SYNC REPLICA without_fixed_size_columns_replica1 PULL; SYSTEM SYNC REPLICA without_fixed_size_columns_replica2; OPTIMIZE TABLE without_fixed_size_columns_replica2 PARTITION 1 FINAL; From 9d0391e6fbc4c88dd68a414f84ae285cb13f53f3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 18:13:57 +0000 Subject: [PATCH 363/473] Add ifndef to fix build --- src/Interpreters/SystemLog.cpp | 1 - src/Loggers/Loggers.cpp | 2 ++ 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3193baa551f..efeb22ba370 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -101,7 +101,6 @@ namespace namespace { -constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; /// Creates a system log with MergeTree engine using parameters from config diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index a9869847f65..4cc74902ee1 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,7 +34,9 @@ static std::string createDirectory(const std::string & file) return path; } +#ifndef WITHOUT_TEXT_LOG constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; +#endif void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { From e0f737f363e08de8c028b0d3eac1908ed3193758 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jul 2023 19:28:35 +0000 Subject: [PATCH 364/473] Remove timeout --- .../02497_trace_events_stress_long.sh | 35 ++++++------------- 1 file changed, 10 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index 3ec729079b8..91f6a9bb541 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -10,7 +10,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function thread1() { thread_id=$1 - while true; do + local TIMELIMIT=$((SECONDS+$2)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do query_id="02497_$CLICKHOUSE_DATABASE-$RANDOM-$thread_id" $CLICKHOUSE_CLIENT --query_id=$query_id --query " SELECT count() FROM numbers_mt(100000) SETTINGS @@ -25,7 +26,8 @@ function thread1() function thread2() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" done } @@ -35,29 +37,12 @@ export -f thread2 TIMEOUT=10 -timeout $TIMEOUT bash -c "thread1 0" >/dev/null & -timeout $TIMEOUT bash -c "thread1 1" >/dev/null & -timeout $TIMEOUT bash -c "thread1 2" >/dev/null & -timeout $TIMEOUT bash -c "thread1 3" >/dev/null & -timeout $TIMEOUT bash -c thread2 >/dev/null & +thread1 0 $TIMEOUT >/dev/null & +thread1 1 $TIMEOUT >/dev/null & +thread1 2 $TIMEOUT >/dev/null & +thread1 3 $TIMEOUT >/dev/null & +thread2 $TIMEOUT >/dev/null & wait -for _ in {1..10} -do - $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null - - # After this moment, the server can still run another query. - # For example, the 'timeout' command killed all threads of thread1, - # and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, - # but just before that, one of the threads successfully sent a query to the server, - # but the server didn't start to run this query yet, - # and even when the KILL QUERY was run, the query from the thread didn't start, - # but only started after the KILL QUERY has been already processed. - - # That's why we have to run this in a loop. - - $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break - - sleep 1 -done +$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' \ No newline at end of file From 9b951e965bd324635435ecb38f14971c6f1944a4 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 19:48:43 +0000 Subject: [PATCH 365/473] Restore define --- src/Interpreters/SystemLog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index efeb22ba370..3193baa551f 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -101,6 +101,7 @@ namespace namespace { +constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; /// Creates a system log with MergeTree engine using parameters from config From a19a1001f063ce6d992ffc08d6d05d3ef7342b66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:28:22 +0200 Subject: [PATCH 366/473] Fix 01111_create_drop_replicated_db_stress --- .../01111_create_drop_replicated_db_stress.sh | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 4d341e5b8a3..cc63af3676b 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -8,7 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function create_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do SHARD=$(($RANDOM % 2)) REPLICA=$(($RANDOM % 2)) SUFFIX=$(($RANDOM % 16)) @@ -24,7 +25,8 @@ function create_db() function drop_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [[ "$database" == "$CLICKHOUSE_DATABASE" ]]; then continue; fi if [ -z "$database" ]; then continue; fi @@ -36,7 +38,8 @@ function drop_db() function sync_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --receive_timeout=1 -q \ @@ -47,7 +50,8 @@ function sync_db() function create_table() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ @@ -59,7 +63,8 @@ function create_table() function alter_table() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ @@ -71,7 +76,8 @@ function alter_table() function insert() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT -q \ @@ -81,23 +87,16 @@ function insert() -export -f create_db -export -f drop_db -export -f sync_db -export -f create_table -export -f alter_table -export -f insert - TIMEOUT=30 -timeout $TIMEOUT bash -c create_db & -timeout $TIMEOUT bash -c sync_db & -timeout $TIMEOUT bash -c create_table & -timeout $TIMEOUT bash -c alter_table & -timeout $TIMEOUT bash -c insert & +create_db $TIMEOUT & +sync_db $TIMEOUT & +create_table $TIMEOUT & +alter_table $TIMEOUT & +insert $TIMEOUT & sleep 1 # give other queries a head start -timeout $TIMEOUT bash -c drop_db & +drop_db $TIMEOUT & wait From 6d915042a23ed0fd320b98118d9527e312d49ffe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:44:20 +0200 Subject: [PATCH 367/473] Fix ugly code --- src/Processors/Formats/Impl/ArrowFieldIndexUtil.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 4beffbcf869..909133dfa4a 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -75,7 +75,7 @@ public: { if (!allow_missing_columns) throw Exception( - ErrorCodes::THERE_IS_NO_COLUMN, "Not found field({}) in arrow schema:{}.", named_col.name, schema.ToString()); + ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the Arrow schema: {}.", named_col.name, schema.ToString()); else continue; } @@ -168,4 +168,3 @@ private: }; } #endif - From 0789f388c3f6acbfdb42f44ee6463b3d646ddc27 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 02:45:56 +0300 Subject: [PATCH 368/473] Update ArrowFieldIndexUtil.h --- src/Processors/Formats/Impl/ArrowFieldIndexUtil.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 909133dfa4a..b7adaa35335 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -75,7 +75,7 @@ public: { if (!allow_missing_columns) throw Exception( - ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the Arrow schema: {}.", named_col.name, schema.ToString()); + ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the following Arrow schema:\n{}\n", named_col.name, schema.ToString()); else continue; } From 65de310137a4e192499119128aa069375eb007c8 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 06:15:57 +0000 Subject: [PATCH 369/473] Return back SystemLogBase --- src/Common/SystemLogBase.cpp | 40 +++++++++++++++++++++++++++++++++- src/Common/SystemLogBase.h | 33 ++++++++++++++++++++++++++++ src/Interpreters/SystemLog.cpp | 29 ++---------------------- src/Interpreters/SystemLog.h | 16 +++----------- 4 files changed, 77 insertions(+), 41 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 1d0673e30dd..baee7021c35 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -202,7 +202,45 @@ void SystemLogQueue::shutdown() flush_event.notify_all(); } -#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; +template +SystemLogBase::SystemLogBase( + const String& name, + size_t flush_interval_milliseconds_, + std::shared_ptr> queue_) + : queue(queue_ ? queue_ : std::make_shared>(name, flush_interval_milliseconds_)) +{ +} + +template +void SystemLogBase::startup() +{ + std::lock_guard lock(queue->mutex); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); +} + +template +void SystemLogBase::add(const LogElement & element) +{ + queue->push(element); +} + +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } + +#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) +#define INSTANTIATE_SYSTEM_LOG_QUEUE(ELEMENT) template class SystemLogQueue; +SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) + } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 9436137d4a8..5718182e115 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -121,4 +121,37 @@ private: const size_t flush_interval_milliseconds; }; + + +template +class SystemLogBase : public ISystemLog +{ +public: + using Self = SystemLogBase; + + SystemLogBase( + const String& name, + size_t flush_interval_milliseconds_, + std::shared_ptr> queue_ = nullptr); + + void startup() override; + + /** Append a record into log. + * Writing to table will be done asynchronously and in case of failure, record could be lost. + */ + void add(const LogElement & element); + + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. + void flush(bool force) override; + + /// Non-blocking flush data in the buffer to disk. + void notifyFlush(bool force); + + String getName() const override { return LogElement::name(); } + + static const char * getDefaultOrderBy() { return "event_date, event_time"; } + +protected: + std::shared_ptr> queue; +}; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3193baa551f..674210cbaad 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -334,23 +334,16 @@ SystemLog::SystemLog( const String & storage_def_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : WithContext(context_) + : Base(database_name_ + "." + table_name_, flush_interval_milliseconds_, queue_) + , WithContext(context_) , log(&Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")")) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) - , queue(queue_ ? queue_ : std::make_shared>(database_name_ + "." + table_name_, flush_interval_milliseconds_)) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); } -template -void SystemLog::startup() -{ - std::lock_guard lock(queue->mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); -} - template void SystemLog::shutdown() { @@ -618,24 +611,6 @@ ASTPtr SystemLog::getCreateTableQuery() return create; } -template -void SystemLog::add(const LogElement & element) -{ - queue->push(element); -} - -template -void SystemLog::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - -template -void SystemLog::notifyFlush(bool force) { queue->notifyFlush(force); } #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 6f61e075b49..91fb7f49221 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -89,10 +89,11 @@ struct SystemLogs template -class SystemLog : public ISystemLog, private boost::noncopyable, WithContext +class SystemLog : public SystemLogBase, private boost::noncopyable, WithContext { public: using Self = SystemLog; + using Base = SystemLogBase; /** Parameter: table name where to write log. * If table is not exists, then it get created with specified engine. @@ -110,23 +111,12 @@ public: size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); - void startup() override; /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. */ - void add(const LogElement & element); void shutdown() override; - String getName() const override { return LogElement::name(); } - static const char * getDefaultOrderBy() { return "event_date, event_time"; } - - /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; - - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - void stopFlushThread() override; protected: @@ -134,6 +124,7 @@ protected: using ISystemLog::is_shutdown; using ISystemLog::saving_thread; + using Base::queue; private: @@ -144,7 +135,6 @@ private: String create_query; String old_create_query; bool is_prepared = false; - std::shared_ptr> queue; /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. From ee0453ed00ab5ecb232557e29d4e1f6365d83cd0 Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:18:50 +0800 Subject: [PATCH 370/473] fix issue#50582 tests Reproduced issue #50582, which occurs when sorting column contains constants and triggers the FinishSortingTransform. --- .../02815_fix_not_found_constants_col_in_block.reference | 2 ++ .../02815_fix_not_found_constants_col_in_block.sql | 5 +++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference create mode 100644 tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference new file mode 100644 index 00000000000..f2d4d23d9e3 --- /dev/null +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference @@ -0,0 +1,2 @@ +\N 1 19000 +\N 1 19000 diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql new file mode 100644 index 00000000000..c56d59c72d6 --- /dev/null +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (vkey UInt32, c0 Float32, primary key(c0)) engine = AggregatingMergeTree; +insert into t0 values (19000, 1); +select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc, c_2_2 asc; +select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc; From d601d86fad94250ca3b749baa4478679cd6e1973 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 07:22:25 +0000 Subject: [PATCH 371/473] Remove empty line --- src/Common/SystemLogBase.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 5718182e115..fa9f9b6f72e 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -122,7 +122,6 @@ private: }; - template class SystemLogBase : public ISystemLog { From 629e0e0269dc96f88f781eb8a0a711667d50c92b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 16:06:21 +0800 Subject: [PATCH 372/473] Fix projection analysis with primary key analysis --- .../QueryPlan/Optimizations/optimizeTree.cpp | 8 +++++--- .../QueryPlan/Optimizations/projectionsCommon.cpp | 3 ++- .../QueryPlan/Optimizations/projectionsCommon.h | 1 - ..._projection_with_query_plan_optimization.reference | 1 + ...normal_projection_with_query_plan_optimization.sql | 11 +++++++++++ 5 files changed, 19 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference create mode 100644 tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 6cb76d540f7..01d192bb1f3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -146,8 +146,13 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } + /// NOTE: optimizePrewhere can modify the stack. + optimizePrewhere(stack, nodes); + optimizePrimaryKeyCondition(stack); + if (optimization_settings.optimize_projection) { + /// Normal projection optimization relies on PK optimization if (optimizeUseNormalProjections(stack, nodes)) { ++num_applied_projection; @@ -164,9 +169,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - /// NOTE: optimizePrewhere can modify the stack. - optimizePrewhere(stack, nodes); - optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*stack.back().node, nodes); stack.pop_back(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index cb76ffa84ba..7ddda29cad4 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -131,7 +131,8 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (prewhere_info->prewhere_actions) { appendExpression(prewhere_info->prewhere_actions); - if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + if (const auto * filter_expression + = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) filter_nodes.push_back(filter_expression); else return false; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 1e9ab67c8fe..35daccad115 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -38,7 +38,6 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea /// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. -/// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. struct QueryDAG { ActionsDAGPtr dag; diff --git a/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql new file mode 100644 index 00000000000..30306ec5637 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql @@ -0,0 +1,11 @@ +drop table if exists t; + +CREATE TABLE t (id UInt64, id2 UInt64, id3 UInt64, PROJECTION t_reverse (SELECT id, id2, id3 ORDER BY id2, id, id3)) ENGINE = MergeTree ORDER BY (id) settings index_granularity = 4; + +insert into t SELECT number, -number, number FROM numbers(10000); + +set max_rows_to_read = 4; + +select count() from t where id = 3; + +drop table t; From 549026f0ae8041ba40f4557922c480f2f07715bf Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Wed, 19 Jul 2023 16:11:14 +0800 Subject: [PATCH 373/473] fix style error fix Trailing whitespaces --- src/Processors/Transforms/FinishSortingTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index dd61472bc37..066928446f2 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -50,7 +50,6 @@ FinishSortingTransform::FinishSortingTransform( description_sorted_without_constants.push_back(column_description); } } - /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. for (const auto & column_sort_desc : description_sorted_without_constants) From a86baab88b4444d5bf34e529bb737817daa20096 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:49:30 +0000 Subject: [PATCH 374/473] Fix test_replicated_database 'node doesn't exist' flakiness --- .../test_replicated_database/test.py | 148 +++++++++--------- 1 file changed, 75 insertions(+), 73 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index b3ba8d4737f..17dd2adcde4 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -132,14 +132,15 @@ def test_create_replicated_table(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_simple_alter_table(started_cluster, engine): + database = f"test_simple_alter_table_{engine}" main_node.query( - "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) # test_simple_alter_table - name = "test_simple_alter_table.alter_test_{}".format(engine) + name = f"{database}.alter_test" main_node.query( "CREATE TABLE {} " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " @@ -187,10 +188,9 @@ def test_simple_alter_table(started_cluster, engine): # test_create_replica_after_delay competing_node.query( - "CREATE DATABASE IF NOT EXISTS test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica3');" + f"CREATE DATABASE IF NOT EXISTS {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica3');" ) - name = "test_simple_alter_table.alter_test_{}".format(engine) main_node.query("ALTER TABLE {} ADD COLUMN Added3 UInt32;".format(name)) main_node.query("ALTER TABLE {} DROP COLUMN AddedNested1;".format(name)) main_node.query("ALTER TABLE {} RENAME COLUMN Added1 TO AddedNested1;".format(name)) @@ -210,21 +210,23 @@ def test_simple_alter_table(started_cluster, engine): ) assert_create_query([main_node, dummy_node, competing_node], name, expected) - main_node.query("DROP DATABASE test_simple_alter_table SYNC") - dummy_node.query("DROP DATABASE test_simple_alter_table SYNC") - competing_node.query("DROP DATABASE test_simple_alter_table SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") + competing_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_delete_from_table(started_cluster, engine): + database = f"delete_from_table_{engine}" + main_node.query( - "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard2', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard2', 'replica1');" ) - name = "delete_from_table.delete_test_{}".format(engine) + name = f"{database}.delete_test" main_node.query( "CREATE TABLE {} " "(id UInt64, value String) " @@ -241,7 +243,7 @@ def test_delete_from_table(started_cluster, engine): table_for_select = name if not "Replicated" in engine: - table_for_select = "cluster('delete_from_table', {})".format(name) + table_for_select = f"cluster('{database}', {name})" for node in [main_node, dummy_node]: assert_eq_with_retry( node, @@ -249,8 +251,8 @@ def test_delete_from_table(started_cluster, engine): expected, ) - main_node.query("DROP DATABASE delete_from_table SYNC") - dummy_node.query("DROP DATABASE delete_from_table SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") def get_table_uuid(database, name): @@ -278,18 +280,18 @@ def fixture_attachable_part(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_attach(started_cluster, attachable_part, engine): + database = f"alter_attach_{engine}" main_node.query( - "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - name = "alter_attach_test_{}".format(engine) main_node.query( - f"CREATE TABLE alter_attach.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_attach_test (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - table_uuid = get_table_uuid("alter_attach", name) + table_uuid = get_table_uuid(database, "alter_attach_test") # Provide and attach a part to the main node shutil.copytree( attachable_part, @@ -298,146 +300,146 @@ def test_alter_attach(started_cluster, attachable_part, engine): f"database/store/{table_uuid[:3]}/{table_uuid}/detached/all_1_1_0", ), ) - main_node.query(f"ALTER TABLE alter_attach.{name} ATTACH PART 'all_1_1_0'") + main_node.query(f"ALTER TABLE {database}.alter_attach_test ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" + assert main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" else: - assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "" - main_node.query("DROP DATABASE alter_attach SYNC") - dummy_node.query("DROP DATABASE alter_attach SYNC") + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "" + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_part(started_cluster, engine): + database = f"alter_drop_part_{engine}" main_node.query( - "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_drop_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_drop_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop_part (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop_part VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (456)") - main_node.query(f"ALTER TABLE alter_drop_part.{table} DROP PART '{part_name}'") - assert main_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" + dummy_node.query(f"INSERT INTO {database}.alter_drop_part VALUES (456)") + main_node.query(f"ALTER TABLE {database}.alter_drop_part DROP PART '{part_name}'") + assert main_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" else: assert ( - dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") + dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "456\n" ) - main_node.query("DROP DATABASE alter_drop_part SYNC") - dummy_node.query("DROP DATABASE alter_drop_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_detach_part(started_cluster, engine): + database = f"alter_detach_part_{engine}" main_node.query( - "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_detach_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_detach_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_detach (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_detach VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (456)") - main_node.query(f"ALTER TABLE alter_detach_part.{table} DETACH PART '{part_name}'") - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_detach_part' AND table='{table}'" + dummy_node.query(f"INSERT INTO {database}.alter_detach VALUES (456)") + main_node.query(f"ALTER TABLE {database}.alter_detach DETACH PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_detach'" assert main_node.query(detached_parts_query) == f"{part_name}\n" if engine == "ReplicatedMergeTree": # The detach operation is still replicated at the table engine level assert dummy_node.query(detached_parts_query) == f"{part_name}\n" else: assert dummy_node.query(detached_parts_query) == "" - main_node.query("DROP DATABASE alter_detach_part SYNC") - dummy_node.query("DROP DATABASE alter_detach_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_detached_part(started_cluster, engine): + database = f"alter_drop_detached_part_{engine}" main_node.query( - "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_drop_detached_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_drop_detached_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop_detached (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop_detached VALUES (123)") main_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DETACH PART '{part_name}'" ) if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (456)") + dummy_node.query(f"INSERT INTO {database}.alter_drop_detached VALUES (456)") dummy_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DETACH PART '{part_name}'" ) main_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DROP DETACHED PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DROP DETACHED PART '{part_name}'" ) - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_drop_detached_part' AND table='{table}'" + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_drop_detached'" assert main_node.query(detached_parts_query) == "" assert dummy_node.query(detached_parts_query) == f"{part_name}\n" - main_node.query("DROP DATABASE alter_drop_detached_part SYNC") - dummy_node.query("DROP DATABASE alter_drop_detached_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_partition(started_cluster, engine): + database = f"alter_drop_partition_{engine}" main_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) snapshotting_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard2', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard2', 'replica1');" ) - table = f"alter_drop_partition.alter_drop_{engine}" main_node.query( - f"CREATE TABLE {table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO {table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO {table} VALUES (456)") - snapshotting_node.query(f"INSERT INTO {table} VALUES (789)") + dummy_node.query(f"INSERT INTO {database}.alter_drop VALUES (456)") + snapshotting_node.query(f"INSERT INTO {database}.alter_drop VALUES (789)") main_node.query( - f"ALTER TABLE {table} ON CLUSTER alter_drop_partition DROP PARTITION ID 'all'", + f"ALTER TABLE {database}.alter_drop ON CLUSTER {database} DROP PARTITION ID 'all'", settings={"replication_alter_partitions_sync": 2}, ) assert ( main_node.query( - f"SELECT CounterID FROM clusterAllReplicas('alter_drop_partition', {table})" + f"SELECT CounterID FROM clusterAllReplicas('{database}', {database}.alter_drop)" ) == "" ) - assert dummy_node.query(f"SELECT CounterID FROM {table}") == "" - main_node.query("DROP DATABASE alter_drop_partition") - dummy_node.query("DROP DATABASE alter_drop_partition") - snapshotting_node.query("DROP DATABASE alter_drop_partition") + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop") == "" + main_node.query(f"DROP DATABASE {database}") + dummy_node.query(f"DROP DATABASE {database}") + snapshotting_node.query(f"DROP DATABASE {database}") def test_alter_fetch(started_cluster): From 8b0fc8283460f5678e733cef9803937ff9913177 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 29 Apr 2023 21:23:55 +0000 Subject: [PATCH 375/473] test_for_basic_auth_registry - mock up --- .../runner/compose/docker_compose_kafka.yml | 10 +++++++++ tests/integration/helpers/cluster.py | 22 ++++++++++++------- .../secrets/password | 1 + .../secrets/schema_registry_jaas.conf | 5 +++++ .../test_format_avro_confluent/test.py | 18 ++++++++++----- 5 files changed, 43 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_format_avro_confluent/secrets/password create mode 100644 tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 7e34f4c114d..39247f1dd37 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -39,7 +39,17 @@ services: environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC + # SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO + # SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein + # SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser + SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar + SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" + volumes: + - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - kafka1 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0614cbf0e0d..9b5b33b1968 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1159,10 +1159,13 @@ class ClickHouseCluster: ] return self.base_kerberized_hdfs_cmd - def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + def setup_kafka_cmd( + self, instance, env_variables, docker_compose_yml_dir + ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) + env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" self.base_cmd.extend( @@ -1498,6 +1501,7 @@ class ClickHouseCluster: with_kafka=False, with_kerberized_kafka=False, with_kerberos_kdc=False, + with_secrets=False, with_rabbitmq=False, with_nats=False, clickhouse_path_dir=None, @@ -1604,6 +1608,7 @@ class ClickHouseCluster: with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, + with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka, with_mongo=with_mongo or with_mongo_secure, with_meili=with_meili, with_redis=with_redis, @@ -3135,6 +3140,7 @@ class ClickHouseInstance: with_nats, with_nginx, with_kerberized_hdfs, + with_secrets, with_mongo, with_meili, with_redis, @@ -3197,7 +3203,7 @@ class ClickHouseInstance: if clickhouse_path_dir else None ) - self.kerberos_secrets_dir = p.abspath(p.join(base_path, "secrets")) + self.secrets_dir = p.abspath(p.join(base_path, "secrets")) self.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path @@ -3220,6 +3226,7 @@ class ClickHouseInstance: self.with_nats = with_nats self.with_nginx = with_nginx self.with_kerberized_hdfs = with_kerberized_hdfs + self.with_secrets = with_secrets self.with_mongo = with_mongo self.with_meili = with_meili self.with_redis = with_redis @@ -4217,17 +4224,16 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - if ( - self.with_kerberized_kafka - or self.with_kerberized_hdfs - or self.with_kerberos_kdc - ): + if self.with_secrets: if self.with_kerberos_kdc: base_secrets_dir = self.cluster.instances_dir else: base_secrets_dir = self.path + from_dir=self.secrets_dir + to_dir=p.abspath(p.join(base_secrets_dir, "secrets")) + logging.debug(f"Copy secret from {from_dir} to {to_dir}") shutil.copytree( - self.kerberos_secrets_dir, + self.secrets_dir, p.abspath(p.join(base_secrets_dir, "secrets")), dirs_exist_ok=True, ) diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password new file mode 100644 index 00000000000..8903cf6edd6 --- /dev/null +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -0,0 +1 @@ +schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user diff --git a/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf b/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf new file mode 100644 index 00000000000..7d0e6e2bf35 --- /dev/null +++ b/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf @@ -0,0 +1,5 @@ +RealmFooBar { + org.eclipse.jetty.jaas.spi.PropertyFileLoginModule required + file="/etc/schema-registry/secrets/password" + debug="true"; +}; \ No newline at end of file diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 42b7ddce193..921fbdf5ef4 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -14,7 +14,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("dummy", with_kafka=True) + cluster.add_instance("dummy", with_kafka=True, with_secrets=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -39,10 +39,13 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None + input("Cluster created, press any key to destroy...") - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(started_cluster.schema_registry_port) - ) + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( @@ -62,7 +65,12 @@ def test_select(started_cluster): data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - schema_registry_url = "http://{}:{}".format( + # schema_registry_url = "http://{}:{}@{}:{}".format( + # 'schemauser', 'letmein', + # started_cluster.schema_registry_host, 8081 + # ) + schema_registry_url = "http://{}:{}@{}:{}".format( + 'schemauser', 'letmein', started_cluster.schema_registry_host, 8081 ) From 1564eace38072417bf2c188d7c0a0c0e55321626 Mon Sep 17 00:00:00 2001 From: dheerajathrey Date: Thu, 7 Jul 2022 13:48:25 +0530 Subject: [PATCH 376/473] enable url-encoded basic auth to fetch avro schema in kafka --- .../Formats/Impl/AvroRowInputFormat.cpp | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 4cd73cb23b5..fe795608970 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -52,6 +52,8 @@ #include #include #include +#include +#include #include #include #include @@ -934,6 +936,29 @@ private: Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(url.getHost()); + if (!url.getUserInfo().empty()) { + Poco::Net::HTTPCredentials http_credentials; + Poco::Net::HTTPBasicCredentials http_basic_credentials; + std::string decoded_username; + std::string decoded_password; + + http_credentials.fromUserInfo(url.getUserInfo()); + + if (!http_credentials.getPassword().empty()) { + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + Poco::URI::decode(http_credentials.getPassword(), decoded_password); + + http_basic_credentials.setUsername(decoded_username); + http_basic_credentials.setPassword(decoded_password); + } + else { + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + http_basic_credentials.setUsername(decoded_username); + } + + http_basic_credentials.authenticate(request); + } + auto session = makePooledHTTPSession(url, timeouts, 1); session->sendRequest(request); From 8e1de7897a0f950a44b9c67b5d7d97b47d380f25 Mon Sep 17 00:00:00 2001 From: dheerajathrey Date: Wed, 24 Aug 2022 19:19:09 +0530 Subject: [PATCH 377/473] indentation fix --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index fe795608970..318ba3cb443 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -936,7 +936,8 @@ private: Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(url.getHost()); - if (!url.getUserInfo().empty()) { + if (!url.getUserInfo().empty()) + { Poco::Net::HTTPCredentials http_credentials; Poco::Net::HTTPBasicCredentials http_basic_credentials; std::string decoded_username; @@ -944,14 +945,16 @@ private: http_credentials.fromUserInfo(url.getUserInfo()); - if (!http_credentials.getPassword().empty()) { + if (!http_credentials.getPassword().empty()) + { Poco::URI::decode(http_credentials.getUsername(), decoded_username); Poco::URI::decode(http_credentials.getPassword(), decoded_password); http_basic_credentials.setUsername(decoded_username); http_basic_credentials.setPassword(decoded_password); } - else { + else + { Poco::URI::decode(http_credentials.getUsername(), decoded_username); http_basic_credentials.setUsername(decoded_username); } From e3523cb1a463931513cb7f3edc9937d64ae82331 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 2 May 2023 12:31:00 +0000 Subject: [PATCH 378/473] test_for_basic_auth_registry - attempt to have two kafka instances --- .../runner/compose/docker_compose_kafka.yml | 44 +++++++++++-- tests/integration/helpers/cluster.py | 58 +++++++++++++++-- .../test_format_avro_confluent/test.py | 65 +++++++++++++++---- 3 files changed, 145 insertions(+), 22 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 39247f1dd37..fc476c09378 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,6 +31,28 @@ services: security_opt: - label:disable + kafka2: + image: confluentinc/cp-kafka:5.2.0 + hostname: kafka2 + ports: + - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} + environment: + # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} + # KAFKA_HOST: ${KAFKA2_HOST} + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 + KAFKA_ADVERTISED_HOST_NAME: kafka2 + KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_BROKER_ID: 2 + KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + depends_on: + - kafka_zookeeper + security_opt: + - label:disable + schema-registry: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry @@ -39,12 +61,24 @@ services: environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + depends_on: + - kafka_zookeeper + - kafka1 + security_opt: + - label:disable + + schema-registry-auth: + image: confluentinc/cp-schema-registry:5.2.0 + hostname: schema-registry-auth + ports: + - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + environment: + SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth + SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka2:19093 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC - # SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO - # SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein - # SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" @@ -52,6 +86,6 @@ services: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - - kafka1 + - kafka2 security_opt: - label:disable diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9b5b33b1968..e0286f6e5c9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -481,12 +481,18 @@ class ClickHouseCluster: # available when with_kafka == True self.kafka_host = "kafka1" + self.kafka2_host = "kafka2" self.kafka_dir = os.path.join(self.instances_dir, "kafka") self._kafka_port = 0 + self._kafka2_port = 0 self.kafka_docker_id = None + self.kafka2_docker_id = None self.schema_registry_host = "schema-registry" self._schema_registry_port = 0 + self.schema_registry_auth_host = "schema-registry-auth" + self._schema_registry_auth_port = 0 self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.kafka2_docker_id = self.get_instance_docker_id(self.kafka2_host) self.coredns_host = "coredns" @@ -650,6 +656,13 @@ class ClickHouseCluster: self._kafka_port = get_free_port() return self._kafka_port + @property + def kafka2_port(self): + if self._kafka2_port: + return self._kafka2_port + self._kafka2_port = get_free_port() + return self._kafka2_port + @property def schema_registry_port(self): if self._schema_registry_port: @@ -657,6 +670,13 @@ class ClickHouseCluster: self._schema_registry_port = get_free_port() return self._schema_registry_port + @property + def schema_registry_auth_port(self): + if self._schema_registry_auth_port: + return self._schema_registry_auth_port + self._schema_registry_auth_port = get_free_port() + return self._schema_registry_auth_port + @property def kerberized_kafka_port(self): if self._kerberized_kafka_port: @@ -1164,10 +1184,13 @@ class ClickHouseCluster: ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host + env_variables["KAFKA2_HOST"] = self.kafka2_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) + env_variables["KAFKA2_EXTERNAL_PORT"] = str(self.kafka2_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" + env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) @@ -2498,20 +2521,44 @@ class ClickHouseCluster: raise Exception("Can't wait Azurite to start") def wait_schema_registry_to_start(self, timeout=180): - sr_client = CachedSchemaRegistryClient( - {"url": "http://localhost:{}".format(self.schema_registry_port)} - ) + reg_url="http://localhost:{}".format(self.schema_registry_port) + arg={'url':reg_url} + sr_client = CachedSchemaRegistryClient(arg) + start = time.time() + sr_started = False + sr_auth_started = False while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) logging.debug("Connected to SchemaRegistry") - return sr_client + sr_started = True + break except Exception as ex: logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) - raise Exception("Can't wait Schema Registry to start") + if not sr_started: + raise Exception("Can't wait Schema Registry to start") + + + auth_reg_url="http://localhost:{}".format(self.schema_registry_auth_port) + auth_arg={'url':auth_reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + + sr_auth_client = CachedSchemaRegistryClient(auth_arg) + while time.time() - start < timeout: + try: + sr_auth_client._send_request(sr_auth_client.url) + logging.debug("Connected to SchemaRegistry with auth") + sr_auth_started = True + break + except Exception as ex: + logging.debug(("Can't connect to SchemaRegistry with auth: %s", str(ex))) + time.sleep(1) + + if not sr_auth_started: + raise Exception("Can't wait Schema Registry with auth to start") def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) @@ -2718,6 +2765,7 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) + self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 921fbdf5ef4..7261ce1b97d 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -37,15 +37,18 @@ def run_query(instance, query, data=None, settings=None): return result + + # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) + # arg={'url':reg_url} + # schema_registry_client = CachedSchemaRegistryClient(arg) + + def test_select(started_cluster): # type: (ClickHouseCluster) -> None - input("Cluster created, press any key to destroy...") - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} - - schema_registry_client = CachedSchemaRegistryClient(arg) + schema_registry_client = CachedSchemaRegistryClient( + "http://localhost:{}".format(started_cluster.schema_registry_port) + ) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( @@ -65,12 +68,7 @@ def test_select(started_cluster): data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - # schema_registry_url = "http://{}:{}@{}:{}".format( - # 'schemauser', 'letmein', - # started_cluster.schema_registry_host, 8081 - # ) - schema_registry_url = "http://{}:{}@{}:{}".format( - 'schemauser', 'letmein', + schema_registry_url = "http://{}:{}".format( started_cluster.schema_registry_host, 8081 ) @@ -83,3 +81,46 @@ def test_select(started_cluster): ["1"], ["2"], ] + + +# def test_select_auth(started_cluster): +# # type: (ClickHouseCluster) -> None + +# reg_url="http://localhost:{}".format( +# started_cluster.schema_registry_auth_port) +# arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + +# schema_registry_client = CachedSchemaRegistryClient(arg) +# serializer = MessageSerializer(schema_registry_client) + +# schema = avro.schema.make_avsc_object( +# { +# "name": "test_record", +# "type": "record", +# "fields": [{"name": "value", "type": "long"}], +# } +# ) + +# buf = io.BytesIO() +# for x in range(0, 3): +# message = serializer.encode_record_with_schema( +# "test_subject", schema, {"value": x} +# ) +# buf.write(message) +# data = buf.getvalue() + +# instance = started_cluster.instances["dummy"] # type: ClickHouseInstance +# schema_registry_url = "http://{}:{}@{}:{}".format( +# 'schemauser', 'letmein', +# started_cluster.schema_registry_auth_host, 8081 +# ) + +# run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") +# settings = {"format_avro_schema_registry_url": schema_registry_url} +# run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) +# stdout = run_query(instance, "select * from avro_data_auth") +# assert list(map(str.split, stdout.splitlines())) == [ +# ["0"], +# ["1"], +# ["2"], +# ] From fb3a860d7f02ddf321875eefefeeaeb46b265bf9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 5 May 2023 10:56:35 +0000 Subject: [PATCH 379/473] test_for_basic_auth_registry - one kafka instance again --- .../runner/compose/docker_compose_kafka.yml | 49 ++++++++++--------- tests/integration/helpers/cluster.py | 2 +- 2 files changed, 27 insertions(+), 24 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index fc476c09378..e0b58fee73d 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,27 +31,27 @@ services: security_opt: - label:disable - kafka2: - image: confluentinc/cp-kafka:5.2.0 - hostname: kafka2 - ports: - - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} - environment: - # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} - # KAFKA_HOST: ${KAFKA2_HOST} - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 - KAFKA_ADVERTISED_HOST_NAME: kafka2 - KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_BROKER_ID: 2 - KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - depends_on: - - kafka_zookeeper - security_opt: - - label:disable + # kafka2: + # image: confluentinc/cp-kafka:5.2.0 + # hostname: kafka2 + # ports: + # - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} + # environment: + # # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} + # # KAFKA_HOST: ${KAFKA2_HOST} + # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 + # KAFKA_ADVERTISED_HOST_NAME: kafka2 + # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 + # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + # KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + # KAFKA_BROKER_ID: 2 + # KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" + # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + # KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + # depends_on: + # - kafka_zookeeper + # security_opt: + # - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 @@ -77,15 +77,18 @@ services: SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka2:19093 + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" + SCHEMA_REGISTRY_GROUP_ID: auth + SCHEMA_REGISTRY_ZK_NAMESPACE: auth + SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemaauth volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - - kafka2 + - kafka1 security_opt: - label:disable diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e0286f6e5c9..e261364ab05 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2765,7 +2765,7 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) + # self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: From db8e96147a9deb92364c8276577dedf68b7653a5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 5 May 2023 17:52:15 +0000 Subject: [PATCH 380/473] test_for_basic_auth_registry - started, but only auth test works --- .../runner/compose/docker_compose_kafka.yml | 94 +++++++++---------- .../test_format_avro_confluent/test.py | 27 ++++-- 2 files changed, 60 insertions(+), 61 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index e0b58fee73d..47e41812cf5 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -1,90 +1,80 @@ version: '2.3' services: + # kafka_zookeeper: + # image: zookeeper:3.4.9 + # hostname: kafka_zookeeper + # environment: + # ZOO_MY_ID: 1 + # ZOO_PORT: 2181 + # ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 + # security_opt: + # - label:disable kafka_zookeeper: - image: zookeeper:3.4.9 - hostname: kafka_zookeeper + image: confluentinc/cp-zookeeper + ports: + - 2181:2181 environment: - ZOO_MY_ID: 1 - ZOO_PORT: 2181 - ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 - security_opt: - - label:disable - + ZOOKEEPER_CLIENT_PORT: 2181 + # security_opt: + # - label:disable kafka1: - image: confluentinc/cp-kafka:5.2.0 + image: confluentinc/cp-kafka hostname: kafka1 ports: - - ${KAFKA_EXTERNAL_PORT:-8081}:${KAFKA_EXTERNAL_PORT:-8081} + - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT + # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_HOST_NAME: kafka1 - KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_BROKER_ID: 1 - KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + # KAFKA_BROKER_ID: 1 + KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 + # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 depends_on: - kafka_zookeeper - security_opt: - - label:disable - - # kafka2: - # image: confluentinc/cp-kafka:5.2.0 - # hostname: kafka2 - # ports: - # - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} - # environment: - # # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} - # # KAFKA_HOST: ${KAFKA2_HOST} - # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 - # KAFKA_ADVERTISED_HOST_NAME: kafka2 - # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 - # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - # KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - # KAFKA_BROKER_ID: 2 - # KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - # KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - # depends_on: - # - kafka_zookeeper - # security_opt: - # - label:disable + # security_opt: + # - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry ports: - - ${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_EXTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry - SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313} depends_on: - kafka_zookeeper - kafka1 - security_opt: - - label:disable + # security_opt: + # - label:disable schema-registry-auth: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry-auth ports: - - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT}:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} environment: - SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + # SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth - SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC - SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: user SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" - SCHEMA_REGISTRY_GROUP_ID: auth - SCHEMA_REGISTRY_ZK_NAMESPACE: auth - SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemaauth + # SCHEMA_REGISTRY_GROUP_ID: auth + SCHEMA_REGISTRY_ZK_NAMESPACE: schema_registry_auth + SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemas2 volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 7261ce1b97d..cd0906bedee 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -1,5 +1,6 @@ import io import logging +import time import avro.schema import pytest @@ -46,14 +47,21 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(started_cluster.schema_registry_port) - ) + time.sleep(3) + + # schema_registry_client = CachedSchemaRegistryClient( + # "http://localhost:{}".format(started_cluster.schema_registry_port) + # ) + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_port) + arg={'url':reg_url} + + schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( { - "name": "test_record", + "name": "test_record1", "type": "record", "fields": [{"name": "value", "type": "long"}], } @@ -62,14 +70,14 @@ def test_select(started_cluster): buf = io.BytesIO() for x in range(0, 3): message = serializer.encode_record_with_schema( - "test_subject", schema, {"value": x} + "test_subject1", schema, {"value": x} ) buf.write(message) data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( - started_cluster.schema_registry_host, 8081 + started_cluster.schema_registry_host, started_cluster.schema_registry_port ) run_query(instance, "create table avro_data(value Int64) engine = Memory()") @@ -85,6 +93,7 @@ def test_select(started_cluster): # def test_select_auth(started_cluster): # # type: (ClickHouseCluster) -> None +# time.sleep(5) # reg_url="http://localhost:{}".format( # started_cluster.schema_registry_auth_port) @@ -95,7 +104,7 @@ def test_select(started_cluster): # schema = avro.schema.make_avsc_object( # { -# "name": "test_record", +# "name": "test_record_auth", # "type": "record", # "fields": [{"name": "value", "type": "long"}], # } @@ -104,7 +113,7 @@ def test_select(started_cluster): # buf = io.BytesIO() # for x in range(0, 3): # message = serializer.encode_record_with_schema( -# "test_subject", schema, {"value": x} +# "test_subject_auth", schema, {"value": x} # ) # buf.write(message) # data = buf.getvalue() @@ -112,7 +121,7 @@ def test_select(started_cluster): # instance = started_cluster.instances["dummy"] # type: ClickHouseInstance # schema_registry_url = "http://{}:{}@{}:{}".format( # 'schemauser', 'letmein', -# started_cluster.schema_registry_auth_host, 8081 +# started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port # ) # run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") From 66581d091af3eda08591e12af551e83a88a95520 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 07:08:04 +0000 Subject: [PATCH 381/473] test_for_basic_auth_registry - both tests works, simplifications --- .../runner/compose/docker_compose_kafka.yml | 46 +++---- tests/integration/helpers/cluster.py | 70 +++------- .../secrets/password | 1 + .../test_format_avro_confluent/test.py | 122 ++++++++++++------ 4 files changed, 116 insertions(+), 123 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 47e41812cf5..5e2e9d87c39 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -1,43 +1,33 @@ version: '2.3' services: - # kafka_zookeeper: - # image: zookeeper:3.4.9 - # hostname: kafka_zookeeper - # environment: - # ZOO_MY_ID: 1 - # ZOO_PORT: 2181 - # ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 - # security_opt: - # - label:disable kafka_zookeeper: - image: confluentinc/cp-zookeeper + image: zookeeper:3.4.9 + hostname: kafka_zookeeper ports: - 2181:2181 environment: - ZOOKEEPER_CLIENT_PORT: 2181 - # security_opt: - # - label:disable + ZOOKEEPER_CLIENT_PORT: 2181 + security_opt: + - label:disable + kafka1: - image: confluentinc/cp-kafka + image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_HOST_NAME: kafka1 - # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT - # KAFKA_BROKER_ID: 1 KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 - # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 depends_on: - kafka_zookeeper - # security_opt: - # - label:disable + security_opt: + - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 @@ -46,15 +36,14 @@ services: - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_EXTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 - SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313} + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT} + SCHEMA_REGISTRY_SCHEMA_REGISTRY_GROUP_ID: noauth depends_on: - kafka_zookeeper - kafka1 - # security_opt: - # - label:disable + security_opt: + - label:disable schema-registry-auth: image: confluentinc/cp-schema-registry:5.2.0 @@ -62,19 +51,14 @@ services: ports: - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT}:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} environment: - # SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC SCHEMA_REGISTRY_AUTHENTICATION_ROLES: user SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" - # SCHEMA_REGISTRY_GROUP_ID: auth - SCHEMA_REGISTRY_ZK_NAMESPACE: schema_registry_auth - SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemas2 + SCHEMA_REGISTRY_SCHEMA_REGISTRY_GROUP_ID: auth volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e261364ab05..c51c97ee6c4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -481,18 +481,14 @@ class ClickHouseCluster: # available when with_kafka == True self.kafka_host = "kafka1" - self.kafka2_host = "kafka2" self.kafka_dir = os.path.join(self.instances_dir, "kafka") self._kafka_port = 0 - self._kafka2_port = 0 self.kafka_docker_id = None - self.kafka2_docker_id = None self.schema_registry_host = "schema-registry" self._schema_registry_port = 0 self.schema_registry_auth_host = "schema-registry-auth" self._schema_registry_auth_port = 0 self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) - self.kafka2_docker_id = self.get_instance_docker_id(self.kafka2_host) self.coredns_host = "coredns" @@ -656,13 +652,6 @@ class ClickHouseCluster: self._kafka_port = get_free_port() return self._kafka_port - @property - def kafka2_port(self): - if self._kafka2_port: - return self._kafka2_port - self._kafka2_port = get_free_port() - return self._kafka2_port - @property def schema_registry_port(self): if self._schema_registry_port: @@ -1184,12 +1173,9 @@ class ClickHouseCluster: ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host - env_variables["KAFKA2_HOST"] = self.kafka2_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) - env_variables["KAFKA2_EXTERNAL_PORT"] = str(self.kafka2_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) - env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] @@ -2521,44 +2507,27 @@ class ClickHouseCluster: raise Exception("Can't wait Azurite to start") def wait_schema_registry_to_start(self, timeout=180): - reg_url="http://localhost:{}".format(self.schema_registry_port) - arg={'url':reg_url} - sr_client = CachedSchemaRegistryClient(arg) + for port in self.schema_registry_port, self.schema_registry_auth_port: + reg_url="http://localhost:{}".format(port) + arg={'url':reg_url} + sr_client = CachedSchemaRegistryClient(arg) - start = time.time() - sr_started = False - sr_auth_started = False - while time.time() - start < timeout: - try: - sr_client._send_request(sr_client.url) - logging.debug("Connected to SchemaRegistry") - sr_started = True - break - except Exception as ex: - logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) - time.sleep(1) + start = time.time() + sr_started = False + sr_auth_started = False + while time.time() - start < timeout: + try: + sr_client._send_request(sr_client.url) + logging.debug("Connected to SchemaRegistry") + # don't care about possible auth errors + sr_started = True + break + except Exception as ex: + logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) + time.sleep(1) - if not sr_started: - raise Exception("Can't wait Schema Registry to start") - - - auth_reg_url="http://localhost:{}".format(self.schema_registry_auth_port) - auth_arg={'url':auth_reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} - - - sr_auth_client = CachedSchemaRegistryClient(auth_arg) - while time.time() - start < timeout: - try: - sr_auth_client._send_request(sr_auth_client.url) - logging.debug("Connected to SchemaRegistry with auth") - sr_auth_started = True - break - except Exception as ex: - logging.debug(("Can't connect to SchemaRegistry with auth: %s", str(ex))) - time.sleep(1) - - if not sr_auth_started: - raise Exception("Can't wait Schema Registry with auth to start") + if not sr_started: + raise Exception("Can't wait Schema Registry to start") def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) @@ -2765,7 +2734,6 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - # self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password index 8903cf6edd6..7fde510bf5a 100644 --- a/tests/integration/test_format_avro_confluent/secrets/password +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -1 +1,2 @@ schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user +schemauser/slash: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index cd0906bedee..d58f6d972d1 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -9,7 +9,7 @@ from confluent_kafka.avro.cached_schema_registry_client import ( ) from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance - +from urllib import parse @pytest.fixture(scope="module") def started_cluster(): @@ -47,11 +47,8 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(3) + # input("Top of test_select, press any key") - # schema_registry_client = CachedSchemaRegistryClient( - # "http://localhost:{}".format(started_cluster.schema_registry_port) - # ) reg_url="http://localhost:{}".format( started_cluster.schema_registry_port) arg={'url':reg_url} @@ -91,45 +88,88 @@ def test_select(started_cluster): ] -# def test_select_auth(started_cluster): -# # type: (ClickHouseCluster) -> None -# time.sleep(5) +def test_select_auth(started_cluster): + # type: (ClickHouseCluster) -> None + time.sleep(5) -# reg_url="http://localhost:{}".format( -# started_cluster.schema_registry_auth_port) -# arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_auth_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} -# schema_registry_client = CachedSchemaRegistryClient(arg) -# serializer = MessageSerializer(schema_registry_client) + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) -# schema = avro.schema.make_avsc_object( -# { -# "name": "test_record_auth", -# "type": "record", -# "fields": [{"name": "value", "type": "long"}], -# } -# ) + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) -# buf = io.BytesIO() -# for x in range(0, 3): -# message = serializer.encode_record_with_schema( -# "test_subject_auth", schema, {"value": x} -# ) -# buf.write(message) -# data = buf.getvalue() + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() -# instance = started_cluster.instances["dummy"] # type: ClickHouseInstance -# schema_registry_url = "http://{}:{}@{}:{}".format( -# 'schemauser', 'letmein', -# started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port -# ) + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + 'schemauser', 'letmein', + started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + ) -# run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") -# settings = {"format_avro_schema_registry_url": schema_registry_url} -# run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) -# stdout = run_query(instance, "select * from avro_data_auth") -# assert list(map(str.split, stdout.splitlines())) == [ -# ["0"], -# ["1"], -# ["2"], -# ] + run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) + stdout = run_query(instance, "select * from avro_data_auth") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] + +def test_select_auth_encoded(started_cluster): + # type: (ClickHouseCluster) -> None + time.sleep(5) + + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_auth_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) + + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth_encoded", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) + + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth_encoded", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() + + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + parse.quote_plus('schemauser/slash'), parse.quote_plus('letmein'), + started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + ) + + run_query(instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()") + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query(instance, "insert into avro_data_auth_encoded format AvroConfluent", data, settings) + stdout = run_query(instance, "select * from avro_data_auth_encoded") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] From 83569688cba16b80ed959c054fd2f36187c520d4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 12:59:08 +0000 Subject: [PATCH 382/473] test_for_basic_auth_registry - UnknownTopicOrPartitionException --- docker/test/integration/runner/compose/docker_compose_kafka.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 5e2e9d87c39..d701af1d425 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -42,6 +42,7 @@ services: depends_on: - kafka_zookeeper - kafka1 + restart: always security_opt: - label:disable @@ -64,5 +65,6 @@ services: depends_on: - kafka_zookeeper - kafka1 + restart: always security_opt: - label:disable From c1c5ffa309c20899f81548bd3314233d84eb03e1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 13:29:19 +0000 Subject: [PATCH 383/473] test_for_basic_auth_registry - cpp code small improvement --- .../Formats/Impl/AvroRowInputFormat.cpp | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 318ba3cb443..a7efc823fbb 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -940,24 +940,19 @@ private: { Poco::Net::HTTPCredentials http_credentials; Poco::Net::HTTPBasicCredentials http_basic_credentials; - std::string decoded_username; - std::string decoded_password; http_credentials.fromUserInfo(url.getUserInfo()); + std::string decoded_username; + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + http_basic_credentials.setUsername(decoded_username); + if (!http_credentials.getPassword().empty()) { - Poco::URI::decode(http_credentials.getUsername(), decoded_username); + std::string decoded_password; Poco::URI::decode(http_credentials.getPassword(), decoded_password); - - http_basic_credentials.setUsername(decoded_username); http_basic_credentials.setPassword(decoded_password); } - else - { - Poco::URI::decode(http_credentials.getUsername(), decoded_username); - http_basic_credentials.setUsername(decoded_username); - } http_basic_credentials.authenticate(request); } From c550a532e649bfde1382bc7f56cac6a38dee0dee Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 20:49:43 +0000 Subject: [PATCH 384/473] test_for_basic_auth_registry - black formatter happy + some doc --- .../operations/settings/settings-formats.md | 11 ++++ tests/integration/helpers/cluster.py | 21 ++++---- .../test_format_avro_confluent/test.py | 54 ++++++++++++------- 3 files changed, 59 insertions(+), 27 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..637ade17296 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1325,6 +1325,17 @@ Default value: 0. Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format. +Format: +``` text +http://[user:password@]machine[:port]" +``` + +Examples: +``` text +http://registry.example.com:8081 +http://admin:secret@registry.example.com:8081 +``` + Default value: `Empty`. ### output_format_avro_codec {#output_format_avro_codec} diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c51c97ee6c4..c52442ecb9c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1168,15 +1168,15 @@ class ClickHouseCluster: ] return self.base_kerberized_hdfs_cmd - def setup_kafka_cmd( - self, instance, env_variables, docker_compose_yml_dir - ): + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) - env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) + env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str( + self.schema_registry_auth_port + ) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) @@ -1617,7 +1617,10 @@ class ClickHouseCluster: with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, - with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka, + with_secrets=with_secrets + or with_kerberized_hdfs + or with_kerberos_kdc + or with_kerberized_kafka, with_mongo=with_mongo or with_mongo_secure, with_meili=with_meili, with_redis=with_redis, @@ -2508,8 +2511,8 @@ class ClickHouseCluster: def wait_schema_registry_to_start(self, timeout=180): for port in self.schema_registry_port, self.schema_registry_auth_port: - reg_url="http://localhost:{}".format(port) - arg={'url':reg_url} + reg_url = "http://localhost:{}".format(port) + arg = {"url": reg_url} sr_client = CachedSchemaRegistryClient(arg) start = time.time() @@ -4245,8 +4248,8 @@ class ClickHouseInstance: base_secrets_dir = self.cluster.instances_dir else: base_secrets_dir = self.path - from_dir=self.secrets_dir - to_dir=p.abspath(p.join(base_secrets_dir, "secrets")) + from_dir = self.secrets_dir + to_dir = p.abspath(p.join(base_secrets_dir, "secrets")) logging.debug(f"Copy secret from {from_dir} to {to_dir}") shutil.copytree( self.secrets_dir, diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index d58f6d972d1..61d839ee63e 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -11,6 +11,7 @@ from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance from urllib import parse + @pytest.fixture(scope="module") def started_cluster(): try: @@ -37,8 +38,6 @@ def run_query(instance, query, data=None, settings=None): return result - - # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) # arg={'url':reg_url} # schema_registry_client = CachedSchemaRegistryClient(arg) @@ -49,9 +48,8 @@ def test_select(started_cluster): # input("Top of test_select, press any key") - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_port) - arg={'url':reg_url} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_port) + arg = {"url": reg_url} schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -92,9 +90,12 @@ def test_select_auth(started_cluster): # type: (ClickHouseCluster) -> None time.sleep(5) - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_auth_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -117,13 +118,17 @@ def test_select_auth(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}@{}:{}".format( - 'schemauser', 'letmein', - started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + "schemauser", + "letmein", + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, ) run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") settings = {"format_avro_schema_registry_url": schema_registry_url} - run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) + run_query( + instance, "insert into avro_data_auth format AvroConfluent", data, settings + ) stdout = run_query(instance, "select * from avro_data_auth") assert list(map(str.split, stdout.splitlines())) == [ ["0"], @@ -131,13 +136,17 @@ def test_select_auth(started_cluster): ["2"], ] + def test_select_auth_encoded(started_cluster): # type: (ClickHouseCluster) -> None time.sleep(5) - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_auth_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -160,13 +169,22 @@ def test_select_auth_encoded(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}@{}:{}".format( - parse.quote_plus('schemauser/slash'), parse.quote_plus('letmein'), - started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + parse.quote_plus("schemauser/slash"), + parse.quote_plus("letmein"), + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, ) - run_query(instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()") + run_query( + instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()" + ) settings = {"format_avro_schema_registry_url": schema_registry_url} - run_query(instance, "insert into avro_data_auth_encoded format AvroConfluent", data, settings) + run_query( + instance, + "insert into avro_data_auth_encoded format AvroConfluent", + data, + settings, + ) stdout = run_query(instance, "select * from avro_data_auth_encoded") assert list(map(str.split, stdout.splitlines())) == [ ["0"], From aa2b7e248d719ed94386e5cf066ed03fe71fef12 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 9 May 2023 08:37:46 +0000 Subject: [PATCH 385/473] test_for_basic_auth_registry - fix port in test_kafka_formats --- .../runner/compose/docker_compose_kafka.yml | 7 ++++--- tests/integration/test_storage_kafka/test.py | 14 +++++++------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index d701af1d425..c0185afb7df 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -17,10 +17,11 @@ services: ports: - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 KAFKA_ADVERTISED_HOST_NAME: kafka1 - KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_BROKER_ID: 1 KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9a6d3e0513c..d0686c7c36f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -762,7 +762,7 @@ def test_kafka_formats(kafka_cluster): ), ], "extra_settings": ", format_avro_schema_registry_url='http://{}:{}'".format( - kafka_cluster.schema_registry_host, 8081 + kafka_cluster.schema_registry_host, kafka_cluster.schema_registry_port ), "supports_empty_value": True, }, @@ -4339,7 +4339,7 @@ def test_row_based_formats(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -4347,10 +4347,10 @@ def test_row_based_formats(kafka_cluster): kafka_group_name = '{format_name}', kafka_format = '{format_name}', kafka_max_rows_per_message = 5; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); """ ) @@ -4459,17 +4459,17 @@ def test_block_based_formats_2(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = '{format_name}', kafka_group_name = '{format_name}', kafka_format = '{format_name}'; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; """ ) From 4259176f24b223decafd0d07bef430a30844e850 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 9 May 2023 09:23:28 +0000 Subject: [PATCH 386/473] test_for_basic_auth_registry - original zk configuration restored --- .../test/integration/runner/compose/docker_compose_kafka.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index c0185afb7df..30d1b0bed3f 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -7,7 +7,9 @@ services: ports: - 2181:2181 environment: - ZOOKEEPER_CLIENT_PORT: 2181 + ZOO_MY_ID: 1 + ZOO_PORT: 2181 + ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 security_opt: - label:disable From f1ce1da00744f17c42d94f1736417474eba478fe Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 18 May 2023 23:21:29 +0000 Subject: [PATCH 387/473] test_for_basic_auth_registry - new test and cleanup per code review --- .../secrets/password | 1 + .../test_format_avro_confluent/test.py | 56 ++++++++++++++++++- 2 files changed, 55 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password index 7fde510bf5a..a367925c806 100644 --- a/tests/integration/test_format_avro_confluent/secrets/password +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -1,2 +1,3 @@ schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user schemauser/slash: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user +complexschemauser: MD5:fcaeda86837fcd37755044e7258edc5d,user diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 61d839ee63e..2d78668f000 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -88,7 +88,6 @@ def test_select(started_cluster): def test_select_auth(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(5) reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) arg = { @@ -139,7 +138,6 @@ def test_select_auth(started_cluster): def test_select_auth_encoded(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(5) reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) arg = { @@ -191,3 +189,57 @@ def test_select_auth_encoded(started_cluster): ["1"], ["2"], ] + +def test_select_auth_encoded_complex(started_cluster): + # type: (ClickHouseCluster) -> None + + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } + + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) + + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth_encoded_complex", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) + + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth_encoded_complex", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() + + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + parse.quote_plus("complexschemauser"), + parse.quote_plus("letmein%@:/"), + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, + ) + + run_query( + instance, "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()" + ) + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query( + instance, + "insert into avro_data_auth_encoded_complex format AvroConfluent", + data, + settings, + ) + stdout = run_query(instance, "select * from avro_data_auth_encoded_complex") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] From 9f6ab5e816378dce815957e396cf4389986256e4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 19 May 2023 09:31:24 +0000 Subject: [PATCH 388/473] test_for_basic_auth_registry - made black formatter happy --- tests/integration/test_format_avro_confluent/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 2d78668f000..b27642c921b 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -190,6 +190,7 @@ def test_select_auth_encoded(started_cluster): ["2"], ] + def test_select_auth_encoded_complex(started_cluster): # type: (ClickHouseCluster) -> None @@ -228,7 +229,8 @@ def test_select_auth_encoded_complex(started_cluster): ) run_query( - instance, "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()" + instance, + "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()", ) settings = {"format_avro_schema_registry_url": schema_registry_url} run_query( From c8347bd31300bec4cdd3277680f398808d37533c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 15:55:44 +0000 Subject: [PATCH 389/473] test_for_basic_auth_registry: some comments removed per code review --- tests/integration/test_format_avro_confluent/test.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index b27642c921b..540f90ae05e 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -38,16 +38,10 @@ def run_query(instance, query, data=None, settings=None): return result - # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) - # arg={'url':reg_url} - # schema_registry_client = CachedSchemaRegistryClient(arg) - def test_select(started_cluster): # type: (ClickHouseCluster) -> None - # input("Top of test_select, press any key") - reg_url = "http://localhost:{}".format(started_cluster.schema_registry_port) arg = {"url": reg_url} From aa888ad64a95ef801977844b2b253bb8162cfc1a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 08:46:57 +0000 Subject: [PATCH 390/473] Separate thread mutex, add test --- src/Common/SystemLogBase.cpp | 2 +- src/Common/SystemLogBase.h | 3 ++- src/Interpreters/SystemLog.cpp | 2 +- src/Interpreters/SystemLog.h | 1 + tests/queries/0_stateless/02813_starting_in_text_log.reference | 1 + tests/queries/0_stateless/02813_starting_in_text_log.sql | 2 ++ 6 files changed, 8 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02813_starting_in_text_log.reference create mode 100755 tests/queries/0_stateless/02813_starting_in_text_log.sql diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index baee7021c35..bed6d661db7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -214,7 +214,7 @@ SystemLogBase::SystemLogBase( template void SystemLogBase::startup() { - std::lock_guard lock(queue->mutex); + std::lock_guard lock(thread_mutex); saving_thread = std::make_unique([this] { savingThreadFunction(); }); } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index fa9f9b6f72e..0ac376769ad 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -67,6 +67,7 @@ public: virtual void savingThreadFunction() = 0; protected: + std::mutex thread_mutex; std::unique_ptr saving_thread; bool is_shutdown = false; @@ -93,10 +94,10 @@ public: Index pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread); void confirm(Index to_flush_end); +private: /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; -private: Poco::Logger * log; // Queue is bounded. But its size is quite large to not block in all normal cases. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 674210cbaad..0b89b1dec26 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -358,7 +358,7 @@ template void SystemLog::stopFlushThread() { { - std::lock_guard lock(queue->mutex); + std::lock_guard lock(thread_mutex); if (!saving_thread || !saving_thread->joinable()) return; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 91fb7f49221..5d8bb30150d 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -124,6 +124,7 @@ protected: using ISystemLog::is_shutdown; using ISystemLog::saving_thread; + using ISystemLog::thread_mutex; using Base::queue; private: diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.reference b/tests/queries/0_stateless/02813_starting_in_text_log.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02813_starting_in_text_log.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql new file mode 100755 index 00000000000..8ef78945a72 --- /dev/null +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -0,0 +1,2 @@ +SYSTEM FLUSH LOGS; +SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Application: Starting ClickHouse%'; From 70543e8ef9fe8523c5604d62fac3376da91c6d2c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 08:47:53 +0000 Subject: [PATCH 391/473] Automatic style fix --- .../test_replicated_database/test.py | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 17dd2adcde4..ed034a326da 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -302,12 +302,21 @@ def test_alter_attach(started_cluster, attachable_part, engine): ) main_node.query(f"ALTER TABLE {database}.alter_attach_test ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" + assert ( + main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "123\n" + ) # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "123\n" + ) else: - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "" + ) main_node.query(f"DROP DATABASE {database} SYNC") dummy_node.query(f"DROP DATABASE {database} SYNC") @@ -333,7 +342,9 @@ def test_alter_drop_part(started_cluster, engine): assert main_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" + ) else: assert ( dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") From 96f048f7f8895507c6827f373699244f345730ec Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 19 Jul 2023 10:46:02 +0200 Subject: [PATCH 392/473] Convert output UInt128 to FixedString even if input is empty --- src/Functions/FunctionsHashing.h | 48 +++++++++---------- .../0_stateless/02534_keyed_siphash.reference | 2 + .../0_stateless/02534_keyed_siphash.sql | 3 ++ 3 files changed, 29 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 4965d1f7b49..82944630b10 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1535,33 +1535,33 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count == 0) - return col_to; - - typename ColumnVector::Container & vec_to = col_to->getData(); - - /// If using a "keyed" algorithm, the first argument is the key and - /// the data starts from the second argument. - /// Otherwise there is no key and all arguments are interpreted as data. - constexpr size_t first_data_argument = Keyed; - - if (arguments.size() <= first_data_argument) + if (input_rows_count != 0) { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } + typename ColumnVector::Container & vec_to = col_to->getData(); - KeyColumnsType key_cols{}; - if constexpr (Keyed) - if (!arguments.empty()) - key_cols = Impl::parseKeyColumns(arguments[0]); + /// If using a "keyed" algorithm, the first argument is the key and + /// the data starts from the second argument. + /// Otherwise there is no key and all arguments are interpreted as data. + constexpr size_t first_data_argument = Keyed; - /// The function supports arbitrary number of arguments of arbitrary types. - bool is_first_argument = true; - for (size_t i = first_data_argument; i < arguments.size(); ++i) - { - const auto & col = arguments[i]; - executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); + if (arguments.size() <= first_data_argument) + { + /// Return a fixed random-looking magic number when input is empty + vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); + } + + KeyColumnsType key_cols{}; + if constexpr (Keyed) + if (!arguments.empty()) + key_cols = Impl::parseKeyColumns(arguments[0]); + + /// The function supports arbitrary number of arguments of arbitrary types. + bool is_first_argument = true; + for (size_t i = first_data_argument; i < arguments.size(); ++i) + { + const auto & col = arguments[i]; + executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); + } } if constexpr (std::is_same_v) /// backward-compatible diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index de783d7dddf..a9f724365a8 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -232,3 +232,5 @@ Check multiple keys as separate ints from a table with constant data 9357996107237883963 86AE90BB6A238D3F6221457630142C9B F6D93D8FEA6D7DECCDD95A7A0A2AA36D +Check asan bug +0 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 14b422ac713..4f3ae7d62bd 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -331,3 +331,6 @@ INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; + +SELECT 'Check asan bug'; +SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806); From f22452c78c36c2d2529444a137d4853749c04945 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 08:54:31 +0000 Subject: [PATCH 393/473] Beautify pretty-printing of the query string in SYSTEM.QUERY_CACHE. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Output of SYSTEM.QUERY_CACHE - before this PR: SELECT * FROM system.query_cache Query id: 4989008b-b84c-4e57-bfe5-7fb551814812 Row 1: ────── query: SELECT 1 SETTINGS [...] - after this PR: SELECT * FROM system.query_cache Query id: 4989008b-b84c-4e57-bfe5-7fb551814812 Row 1: ────── query: SELECT 1 SETTINGS use_query_cache = 1 [...] --- src/Interpreters/Cache/QueryCache.cpp | 35 ++++++++++--------- src/Interpreters/Cache/QueryCache.h | 8 +++-- .../System/StorageSystemQueryCache.cpp | 2 +- .../02494_query_cache_secrets.reference | 2 +- 4 files changed, 26 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index f46a10ca51d..1d1543844a2 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -115,6 +115,15 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } +String queryStringFromAst(ASTPtr ast) +{ + WriteBufferFromOwnString buf; + IAST::FormatSettings format_settings(buf, /*one_line*/ true); + format_settings.show_secrets = false; + ast->format(format_settings); + return buf.str(); +} + } QueryCache::Key::Key( @@ -129,6 +138,7 @@ QueryCache::Key::Key( , is_shared(is_shared_) , expires_at(expires_at_) , is_compressed(is_compressed_) + , query_string(queryStringFromAst(ast_)) { } @@ -142,15 +152,6 @@ bool QueryCache::Key::operator==(const Key & other) const return ast->getTreeHash() == other.ast->getTreeHash(); } -String QueryCache::Key::queryStringFromAst() const -{ - WriteBufferFromOwnString buf; - IAST::FormatSettings format_settings(buf, /*one_line*/ true); - format_settings.show_secrets = false; - ast->format(format_settings); - return buf.str(); -} - size_t QueryCache::KeyHasher::operator()(const Key & key) const { SipHash hash; @@ -191,7 +192,7 @@ QueryCache::Writer::Writer( if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { skip_insert = true; /// Key already contained in cache and did not expire yet --> don't replace it - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.query_string); } } @@ -263,14 +264,14 @@ void QueryCache::Writer::finalizeWrite() if (std::chrono::duration_cast(std::chrono::system_clock::now() - query_start_time) < min_query_runtime) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.query_string); return; } if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { /// Same check as in ctor because a parallel Writer could have inserted the current key in the meantime - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.query_string); return; } @@ -353,7 +354,7 @@ void QueryCache::Writer::finalizeWrite() if ((new_entry_size_in_bytes > max_entry_size_in_bytes) || (new_entry_size_in_rows > max_entry_size_in_rows)) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.query_string); return; } @@ -388,7 +389,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar if (!entry.has_value()) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.query_string); return; } @@ -397,13 +398,13 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar if (!entry_key.is_shared && entry_key.user_name != key.user_name) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.query_string); return; } if (IsStale()(entry_key)) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.query_string); return; } @@ -441,7 +442,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes); } - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.query_string); } bool QueryCache::Reader::hasCacheEntryForKey() const diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 6ef7cc60918..a67adcc86c9 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -30,7 +30,7 @@ public: /// ---------------------------------------------------- /// The actual key (data which gets hashed): - /// Unlike the query string, the AST is agnostic to lower/upper case (SELECT vs. select) + /// Unlike the query string, the AST is agnostic to lower/upper case (SELECT vs. select). const ASTPtr ast; /// Note: For a transactionally consistent cache, we would need to include the system settings in the cache key or invalidate the @@ -58,6 +58,11 @@ public: /// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case) const bool is_compressed; + /// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the + /// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in + /// the SYSTEM.QUERY_CACHE. + const String query_string; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, Block header_, @@ -69,7 +74,6 @@ public: Key(ASTPtr ast_, const String & user_name_); bool operator==(const Key & other) const; - String queryStringFromAst() const; }; struct Entry diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 117fb4e8a5c..288e4fd52a0 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -44,7 +44,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr if (!key.is_shared && key.user_name != user_name) continue; - res_columns[0]->insert(key.queryStringFromAst()); /// approximates the original query string + res_columns[0]->insert(key.query_string); /// approximates the original query string res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result)); res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); res_columns[3]->insert(key.is_shared); diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.reference b/tests/queries/0_stateless/02494_query_cache_secrets.reference index dd6341262bc..306374eed4b 100644 --- a/tests/queries/0_stateless/02494_query_cache_secrets.reference +++ b/tests/queries/0_stateless/02494_query_cache_secrets.reference @@ -1,2 +1,2 @@ A2193552DCF8A9F99AC35F86BC4D2FFD -SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS +SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS use_query_cache = 1 From d3663c356f4a8cab8b77f36fcc9b8251e6ddd02e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 19 Jul 2023 12:11:57 +0200 Subject: [PATCH 394/473] Wait fo KILL MUTATION to finish --- .../00834_kill_mutation_replicated_zookeeper.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 2e917f67fe8..16ad08deeb2 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -57,6 +57,14 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM system.mutations WHERE database ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" +# Wait for the 1st mutation to be actually killed and the 2nd to finish +query_result=$($CLICKHOUSE_CLIENT --query="$check_query1" 2>&1) +while [ "$query_result" != "0" ] +do + query_result=$($CLICKHOUSE_CLIENT --query="$check_query1" 2>&1) + sleep 0.5 +done + ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r2" From 95424177d5de5bd7973823ffdaaacafce442e8ba Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 18:26:54 +0800 Subject: [PATCH 395/473] review fix --- docs/en/interfaces/formats.md | 3 +- .../operations/settings/settings-formats.md | 12 +-- src/Core/Settings.h | 3 +- src/Formats/FormatFactory.cpp | 3 +- src/Formats/FormatSettings.h | 3 +- .../Formats/Impl/CSVRowInputFormat.cpp | 78 ++++++++++--------- .../Formats/Impl/CSVRowInputFormat.h | 2 + ...11_csv_input_field_type_mismatch.reference | 8 +- .../02811_csv_input_field_type_mismatch.sh | 4 +- .../data_csv/csv_with_bad_field_values.csv | 5 ++ .../data_csv/csv_with_diff_field_types.csv | 3 - 11 files changed, 62 insertions(+), 62 deletions(-) create mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv delete mode 100644 tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index f45c55a9734..c20f304c346 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_field_deserialization](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_field_deserialization) - Allow to check whether the csv input field can be successful deserialized. Default value - `false`. -- [input_format_csv_set_default_if_deserialization_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialization_failed) - Set default value to column if the csv input field deserialization failed. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialize failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 409ac4bd58a..5fac8df02d7 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,15 +969,9 @@ Result a b ``` -### input_format_csv_allow_check_field_deserialization {#input_format_csv_allow_check_field_deserialization} +### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} -Allow to use whitespace or tab as field delimiter in CSV strings. - -Default value: `false`. - -### input_format_csv_set_default_if_deserialization_failed {#input_format_csv_set_default_if_deserialization_failed} - -Allow to set default value to column if the csv input field's deserialization failed +Allow to set default value to column when CSV field deserialize failed on bad value Default value: `false`. @@ -988,7 +982,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_field_deserialization=true, input_format_csv_set_default_if_deserialization_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9d93ba9ad2c..311813fb38f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_field_deserialization, false, "Allow to check the csv input field deserialization whether success or not.", 0) \ - M(Bool, input_format_csv_set_default_if_deserialization_failed, false, "All to set column default value if the input field's deserialization failed.", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialize failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b3b9609f9fe..3df2ca7d2e0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,8 +73,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.allow_check_field_deserialization = settings.input_format_csv_allow_check_field_deserialization; - format_settings.csv.set_default_if_deserialization_failed = settings.input_format_csv_set_default_if_deserialization_failed; + format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 4d4eb926992..4e49d338e43 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,7 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_field_deserialization=false; - bool set_default_if_deserialization_failed=false; + bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index e1be6b21610..34d5b589591 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -316,49 +317,52 @@ bool CSVFormatReader::readField( return false; } - BufferBase::Position pos_start = buf->position(); + if (format_settings.csv.use_default_on_bad_values) + return readFieldOrDefault(column, type, serialization); + return readFieldImpl(*buf, column, type, serialization); +} + +bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + { + /// If value is null but type is not nullable then use default value instead. + return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + } + + /// Read the column normally. + serialization->deserializeTextCSV(column, istr, format_settings); + return true; +} + +bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + String field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromString tmp_buf(field); + bool is_bad_value = false; + bool res = false; + size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_field_deserialization) - { - std::string field; - readCSVField(field, *buf, format_settings.csv); - ReadBufferFromMemory tmp(field); - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); - else - serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && !tmp.eof()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); - } - else - { - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - { - /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); - } - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); - } + res = readFieldImpl(tmp_buf, column, type, serialization); + /// Check if we parsed the whole field successfully. + if (!field.empty() && !tmp_buf.eof()) + is_bad_value = true; } - catch (Exception & e) + catch (const Exception &) { - LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.set_default_if_deserialization_failed) - { - // Reset the column and buffer position, then skip the field and set column default value. - if (column.size() == col_size + 1) - column.popBack(1); - buf->position() = pos_start; - skipField(); - column.insertDefault(); - } - else - throw; + is_bad_value = true; } - return true; + + if (!is_bad_value) + return res; + + if (column.size() == col_size + 1) + column.popBack(1); + column.insertDefault(); + return false; } void CSVFormatReader::skipPrefixBeforeHeader() diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 8ccf04feed3..7b1a1fc433d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } + bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); + bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference index c5ee611a230..19c7956ba84 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -1,3 +1,5 @@ -a 1 2023-03-14 -a 0 1970-01-01 -c 1 1970-01-01 +0 111 1970-01-01 2023-03-24 00:00:00 false +1 abc 2023-03-14 2023-03-14 11:22:33 true +2 c 1970-01-01 1970-01-01 08:00:00 false +4 888 2023-03-14 1970-06-03 14:43:53 false +5 bks 1970-01-01 2023-07-19 18:17:59 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index df736ea6792..3961664b9b3 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, d DateTime, e Boolean) engine=MergeTree order by a" +cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv new file mode 100644 index 00000000000..faedd9b6705 --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -0,0 +1,5 @@ +1,abc,2023-03-14,2023-03-14 11:22:33,true +2,c,ab,2023,false +bc,111,ab,2023-03-24,ban +4,888,2023-03-14,13243433,false +5,bks,2023-03,1689761879,abdd \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv deleted file mode 100644 index 464172c515c..00000000000 --- a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv +++ /dev/null @@ -1,3 +0,0 @@ -a,1,2023-03-14 -a,b,c -c,1,a \ No newline at end of file From 380b4ffe2be4107ae3965cba19c5b697e7108128 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 12:29:39 +0200 Subject: [PATCH 396/473] Reduce dependencies for skim by avoid using default features By default skim requires cli -> clap -> termcolor -> winapi-util Signed-off-by: Azat Khuzhin --- rust/skim/Cargo.lock | 204 ++++++++++++------------------------------- rust/skim/Cargo.toml | 2 +- 2 files changed, 58 insertions(+), 148 deletions(-) diff --git a/rust/skim/Cargo.lock b/rust/skim/Cargo.lock index 9f948ee1c38..f55ea8a84b0 100644 --- a/rust/skim/Cargo.lock +++ b/rust/skim/Cargo.lock @@ -42,17 +42,6 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi 0.1.19", - "libc", - "winapi", -] - [[package]] name = "autocfg" version = "1.1.0" @@ -104,31 +93,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "clap" -version = "3.2.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ea181bf566f71cb9a5d17a59e1871af638180a18fb0035c92ae62b705207123" -dependencies = [ - "atty", - "bitflags", - "clap_lex", - "indexmap", - "once_cell", - "strsim", - "termcolor", - "textwrap", -] - -[[package]] -name = "clap_lex" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2850f2f5a82cbf437dd5af4d49848fbdfc27c157c3d010345776f952765261c5" -dependencies = [ - "os_str_bytes", -] - [[package]] name = "codespan-reporting" version = "0.11.1" @@ -214,9 +178,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88abab2f5abbe4c56e8f1fb431b784d710b709888f35755a160e62e33fe38e8" +checksum = "5032837c1384de3708043de9d4e97bb91290faca6c16529a28aa340592a78166" dependencies = [ "cc", "cxxbridge-flags", @@ -226,9 +190,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c0c11acd0e63bae27dcd2afced407063312771212b7a823b4fd72d633be30fb" +checksum = "51368b3d0dbf356e10fcbfd455a038503a105ee556f7ee79b6bb8c53a7247456" dependencies = [ "cc", "codespan-reporting", @@ -236,24 +200,24 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] name = "cxxbridge-flags" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d3816ed957c008ccd4728485511e3d9aaf7db419aa321e3d2c5a2f3411e36c8" +checksum = "0d9062157072e4aafc8e56ceaf8325ce850c5ae37578c852a0d4de2cecdded13" [[package]] name = "cxxbridge-macro" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a26acccf6f445af85ea056362561a24ef56cdc15fcc685f03aec50b9c702cb6d" +checksum = "cf01e8a540f5a4e0f284595834f81cf88572f244b768f051724537afa99a2545" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] @@ -359,19 +323,6 @@ version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" -[[package]] -name = "env_logger" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12e6657c4c97ebab115a42dcee77225f7f482cdd841cf7088c657a42e9e00e7" -dependencies = [ - "atty", - "humantime", - "log", - "regex", - "termcolor", -] - [[package]] name = "fnv" version = "1.0.7" @@ -398,32 +349,11 @@ dependencies = [ "wasi 0.11.0+wasi-snapshot-preview1", ] -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" - [[package]] name = "hermit-abi" -version = "0.1.19" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - -[[package]] -name = "hermit-abi" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" - -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" +checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" [[package]] name = "iana-time-zone" @@ -454,16 +384,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" -[[package]] -name = "indexmap" -version = "1.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" -dependencies = [ - "autocfg", - "hashbrown", -] - [[package]] name = "js-sys" version = "0.3.64" @@ -487,9 +407,9 @@ checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" [[package]] name = "link-cplusplus" -version = "1.0.8" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" +checksum = "9d240c6f7e1ba3a28b0249f774e6a9dd0175054b52dfbb61b16eb8505c3785c9" dependencies = [ "cc", ] @@ -564,7 +484,7 @@ version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi 0.3.1", + "hermit-abi", "libc", ] @@ -574,12 +494,6 @@ version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" -[[package]] -name = "os_str_bytes" -version = "6.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" - [[package]] name = "pin-utils" version = "0.1.0" @@ -588,18 +502,18 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "proc-macro2" -version = "1.0.63" +version = "1.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +checksum = "18fb31db3f9bddb2ea821cde30a9f70117e3f119938b5ee630b7403aa6e2ead9" dependencies = [ "unicode-ident", ] [[package]] name = "quote" -version = "1.0.29" +version = "1.0.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" +checksum = "5fe8a65d69dd0808184ebb5f836ab526bb259db23c657efa38711b1072ee47f0" dependencies = [ "proc-macro2", ] @@ -648,9 +562,21 @@ dependencies = [ [[package]] name = "regex" -version = "1.8.4" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0ab3ca65655bb1e41f2a8c8cd662eb4fb035e67c3f78da1d61dffe89d07300f" +checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310" dependencies = [ "aho-corasick", "memchr", @@ -659,39 +585,33 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.7.2" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" +checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" [[package]] name = "rustversion" -version = "1.0.12" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" [[package]] name = "scopeguard" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "scratch" -version = "1.0.5" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" +checksum = "a3cf7c11c38cb994f3d40e8a8cde3bbd1f72a435e4c49e85d6553d8312306152" [[package]] name = "serde" -version = "1.0.164" +version = "1.0.171" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e8c8cf938e98f769bc164923b06dce91cea1751522f46f8466461af04c9027d" - -[[package]] -name = "shlex" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43b2853a4d09f215c24cc5489c992ce46052d359b5109343cbafbf26bc62f8a3" +checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9" [[package]] name = "skim" @@ -699,23 +619,19 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5d28de0a6cb2cdd83a076f1de9d965b973ae08b244df1aa70b432946dda0f32" dependencies = [ - "atty", "beef", "bitflags", "chrono", - "clap", "crossbeam", "defer-drop", "derive_builder", - "env_logger", "fuzzy-matcher", "lazy_static", "log", "nix 0.25.1", "rayon", "regex", - "shlex", - "time 0.3.22", + "time 0.3.23", "timer", "tuikit", "unicode-width", @@ -741,9 +657,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.23" +version = "2.0.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +checksum = "45c3457aacde3c65315de5031ec191ce46604304d2446e803d71ade03308d970" dependencies = [ "proc-macro2", "quote", @@ -770,30 +686,24 @@ dependencies = [ "winapi-util", ] -[[package]] -name = "textwrap" -version = "0.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" - [[package]] name = "thiserror" -version = "1.0.40" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "978c9a314bd8dc99be594bc3c175faaa9794be04a5a5e153caba6915336cebac" +checksum = "a35fc5b8971143ca348fa6df4f024d4d55264f3468c71ad1c2f365b0a4d58c42" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.40" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" +checksum = "463fe12d7993d3b327787537ce8dd4dfa058de32fc2b195ef3cde03dc4771e8f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] @@ -819,9 +729,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.22" +version = "0.3.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea9e1b3cf1243ae005d9e74085d4d542f3125458f3a81af210d901dcd7411efd" +checksum = "59e399c068f43a5d116fedaf73b203fa4f9c519f17e2b34f63221d3792f81446" dependencies = [ "serde", "time-core", @@ -858,9 +768,9 @@ dependencies = [ [[package]] name = "unicode-ident" -version = "1.0.9" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15811caf2415fb889178633e7724bad2509101cde276048e013b9def5e51fa0" +checksum = "301abaae475aa91687eb82514b328ab47a211a533026cb25fc3e519b86adfc3c" [[package]] name = "unicode-width" @@ -928,7 +838,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", "wasm-bindgen-shared", ] @@ -950,7 +860,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", "wasm-bindgen-backend", "wasm-bindgen-shared", ] diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index e5801a26f77..0381ad81619 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -6,7 +6,7 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -skim = "0.10.2" +skim = { version = "0.10.2", default-features = false } cxx = "1.0.83" term = "0.7.0" From af6361e2a0c78f45500a37bc67f563bd74412076 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 19 Jul 2023 12:35:52 +0200 Subject: [PATCH 397/473] Fix 02725_memory-for-merges --- tests/queries/0_stateless/02725_memory-for-merges.sql | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02725_memory-for-merges.sql b/tests/queries/0_stateless/02725_memory-for-merges.sql index 347c8b2a8d3..1a8402dff4b 100644 --- a/tests/queries/0_stateless/02725_memory-for-merges.sql +++ b/tests/queries/0_stateless/02725_memory-for-merges.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, no-random-merge-tree-settings -- We allocate a lot of memory for buffers when reading or writing to S3 DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; @@ -21,7 +21,6 @@ OPTIMIZE TABLE 02725_memory_for_merges FINAL; SYSTEM FLUSH LOGS; -WITH (SELECT uuid FROM system.tables WHERE table='02725_memory_for_merges' and database=currentDatabase()) as uuid -SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where table_uuid=uuid and event_type='MergeParts'; +SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where database=currentDatabase() and table='02725_memory_for_merges' and event_type='MergeParts'; DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; From 2ebbbf0000ce7f5767d754b0aee777a4255ab7b3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 19:13:45 +0800 Subject: [PATCH 398/473] Also need to fix aggregate projections --- .../QueryPlan/Optimizations/optimizeTree.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 01d192bb1f3..b13dda9a8f0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -114,6 +114,10 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s while (!stack.empty()) { + /// NOTE: optimizePrewhere can modify the stack. + optimizePrewhere(stack, nodes); + optimizePrimaryKeyCondition(stack); + { /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); @@ -125,6 +129,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); + /// Projection optimization relies on PK optimization if (optimization_settings.optimize_projection) num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); @@ -146,13 +151,9 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - /// NOTE: optimizePrewhere can modify the stack. - optimizePrewhere(stack, nodes); - optimizePrimaryKeyCondition(stack); - if (optimization_settings.optimize_projection) { - /// Normal projection optimization relies on PK optimization + /// Projection optimization relies on PK optimization if (optimizeUseNormalProjections(stack, nodes)) { ++num_applied_projection; From 94796f28adcd5b304b9fbc8a715462f4cfb1c1fd Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 19:24:16 +0800 Subject: [PATCH 399/473] ci fix --- docs/en/interfaces/formats.md | 2 +- docs/en/operations/settings/settings-formats.md | 2 +- src/Core/Settings.h | 2 +- .../queries/0_stateless/02811_csv_input_field_type_mismatch.sh | 0 4 files changed, 3 insertions(+), 3 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c20f304c346..ddf4ab3f78e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,7 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialize failed on bad value. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 5fac8df02d7..fb04ac23d3a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -971,7 +971,7 @@ a b ### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} -Allow to set default value to column when CSV field deserialize failed on bad value +Allow to set default value to column when CSV field deserialization failed on bad value Default value: `false`. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 311813fb38f..309dfe0d2ec 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,7 +872,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialize failed on bad value", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh old mode 100644 new mode 100755 From 7837559dbfdc194f28681dda808bc06b6609dd8b Mon Sep 17 00:00:00 2001 From: Song Liyong Date: Wed, 12 Jul 2023 17:13:04 +0200 Subject: [PATCH 400/473] MaterializedMySQL: Support CREATE TABLE AS SELECT --- src/Core/MySQL/MySQLReplication.cpp | 11 +++++++ .../materialized_with_ddl.py | 29 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 6 ++++ 3 files changed, 46 insertions(+) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1ee027b7185..ab4a37d2466 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -121,6 +121,17 @@ namespace MySQLReplication { typ = QUERY_SAVEPOINT; } + + // https://dev.mysql.com/worklog/task/?id=13355 + // When doing query "CREATE TABLE xx AS SELECT", the binlog will be + // "CREATE TABLE ... START TRANSACTION", the DDL will be failed + // so, just ignore the "START TRANSACTION" suffix + if (query.ends_with("START TRANSACTION")) + { + auto pos = query.rfind("START TRANSACTION"); + if (pos > 0) + query.resize(pos); + } } void QueryEvent::dump(WriteBuffer & out) const diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8cf9e67bf63..60326e422c9 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2336,3 +2336,32 @@ def named_collections(clickhouse_node, mysql_node, service_name): ) clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + + +def create_table_as_select(clickhouse_node, mysql_node, service_name): + db = "create_table_as_select" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializeMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + mysql_node.query( + f"CREATE TABLE {db}.t1(a INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query(f"INSERT INTO {db}.t1 VALUES (1)") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\n", + ) + + mysql_node.query(f"CREATE TABLE {db}.t2(PRIMARY KEY(a)) AS SELECT * FROM {db}.t1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\nt2\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 21316d1a474..f227c19e6b8 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -529,3 +529,9 @@ def test_named_collections(started_cluster, started_mysql_8_0, clickhouse_node): materialized_with_ddl.named_collections( clickhouse_node, started_mysql_8_0, "mysql80" ) + + +def test_create_table_as_select(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.create_table_as_select( + clickhouse_node, started_mysql_8_0, "mysql80" + ) From dcf7ba25348f88bda0ef144ce068cc9005cb3ada Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 19:36:19 +0800 Subject: [PATCH 401/473] remove unuseful code --- docs/en/operations/settings/settings-formats.md | 3 +-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index fb04ac23d3a..b3bc3afafd3 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -980,9 +980,8 @@ Default value: `false`. Query ```bash -echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 34d5b589591..244b906549e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -12,7 +12,6 @@ #include #include #include -#include namespace DB From cc9da46efa2af4dfd4f8dfdfa84327f5f14a8630 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 20:11:03 +0800 Subject: [PATCH 402/473] ci fix --- .../02811_csv_input_field_type_mismatch.reference | 10 +++++----- .../0_stateless/02811_csv_input_field_type_mismatch.sh | 2 +- .../0_stateless/data_csv/csv_with_bad_field_values.csv | 10 +++++----- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference index 19c7956ba84..6abcc56bacc 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -1,5 +1,5 @@ -0 111 1970-01-01 2023-03-24 00:00:00 false -1 abc 2023-03-14 2023-03-14 11:22:33 true -2 c 1970-01-01 1970-01-01 08:00:00 false -4 888 2023-03-14 1970-06-03 14:43:53 false -5 bks 1970-01-01 2023-07-19 18:17:59 false +0 111 1970-01-01 false +1 abc 2023-03-14 true +2 c 1970-01-01 false +4 888 2023-03-14 false +5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index 3961664b9b3..30223329eca 100755 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, d DateTime, e Boolean) engine=MergeTree order by a" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv index faedd9b6705..e829cc0106a 100644 --- a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -1,5 +1,5 @@ -1,abc,2023-03-14,2023-03-14 11:22:33,true -2,c,ab,2023,false -bc,111,ab,2023-03-24,ban -4,888,2023-03-14,13243433,false -5,bks,2023-03,1689761879,abdd \ No newline at end of file +1,abc,2023-03-14,true +2,c,ab,false +bc,111,ab,ban +4,888,2023-03-14,false +5,bks,2023-03,abdd \ No newline at end of file From 0c86df519ffa8921b6c546b304705366838dfa21 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Jul 2023 12:41:25 +0000 Subject: [PATCH 403/473] Fix unspported disks in Keeper --- src/Coordination/KeeperContext.cpp | 31 ++++++++++++++++++- src/Disks/DiskSelector.cpp | 5 ++- src/Disks/DiskSelector.h | 3 +- .../configs/enable_keeper.xml | 4 +++ tests/integration/test_keeper_disks/test.py | 12 ++++++- 5 files changed, 51 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 408344ee67f..32f8b98a7ed 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -41,9 +41,38 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) initializeDisks(config); } +namespace +{ + +bool diskValidator(const Poco::Util::AbstractConfiguration & config, const std::string & disk_config_prefix) +{ + const auto disk_type = config.getString(disk_config_prefix + ".type", "local"); + + using namespace std::literals; + static constexpr std::array supported_disk_types + { + "s3"sv, + "s3_plain"sv, + "local"sv + }; + + if (std::all_of( + supported_disk_types.begin(), + supported_disk_types.end(), + [&](const auto supported_type) { return disk_type != supported_type; })) + { + LOG_INFO(&Poco::Logger::get("KeeperContext"), "Disk type '{}' is not supported for Keeper", disk_type); + return false; + } + + return true; +} + +} + void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & config) { - disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); + disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance(), diskValidator); log_storage = getLogsPathFromConfig(config); diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index e51f79867b5..415e10a55fc 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -27,7 +27,7 @@ void DiskSelector::assertInitialized() const } -void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); @@ -46,6 +46,9 @@ void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, auto disk_config_prefix = config_prefix + "." + disk_name; + if (disk_validator && !disk_validator(config, disk_config_prefix)) + continue; + disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context, disks)); } if (!has_default_disk) diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 58adeb953db..c91c3acb3bd 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -23,7 +23,8 @@ public: DiskSelector() = default; DiskSelector(const DiskSelector & from) = default; - void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); + using DiskValidator = std::function; + void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator = {}); DiskSelectorPtr updateFromConfig( const Poco::Util::AbstractConfiguration & config, diff --git a/tests/integration/test_keeper_disks/configs/enable_keeper.xml b/tests/integration/test_keeper_disks/configs/enable_keeper.xml index 5814979229c..50d0329637a 100644 --- a/tests/integration/test_keeper_disks/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_disks/configs/enable_keeper.xml @@ -1,6 +1,10 @@ + + hdfs + hdfs://hdfs1:9000/ + local /var/lib/clickhouse/coordination/logs/ diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py index 11bb215be54..86682bcde01 100644 --- a/tests/integration/test_keeper_disks/test.py +++ b/tests/integration/test_keeper_disks/test.py @@ -9,7 +9,11 @@ import os CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_minio=True + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, + with_minio=True, + with_hdfs=True, ) from kazoo.client import KazooClient, KazooState @@ -117,6 +121,12 @@ def get_local_snapshots(): return get_local_files("/var/lib/clickhouse/coordination/snapshots") +def test_supported_disk_types(started_cluster): + node.stop_clickhouse() + node.start_clickhouse() + node.contains_in_log("Disk type 'hdfs' is not supported for Keeper") + + def test_logs_with_disks(started_cluster): setup_local_storage(started_cluster) From 7b3564f96aa44bde8aa33914930ca3bbf1c5f52e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:44:59 +0200 Subject: [PATCH 404/473] Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed" --- docs/en/interfaces/formats.md | 1 - .../operations/settings/settings-formats.md | 22 ---------- src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - .../Formats/Impl/CSVRowInputFormat.cpp | 42 +------------------ .../Formats/Impl/CSVRowInputFormat.h | 2 - ...11_csv_input_field_type_mismatch.reference | 5 --- .../02811_csv_input_field_type_mismatch.sh | 13 ------ .../data_csv/csv_with_bad_field_values.csv | 5 --- 10 files changed, 2 insertions(+), 91 deletions(-) delete mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference delete mode 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh delete mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ddf4ab3f78e..ed2f010a632 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,7 +472,6 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index c8adc83d3ad..0915c51806a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -989,28 +989,6 @@ Result a b ``` -### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} - -Allow to set default value to column when CSV field deserialization failed on bad value - -Default value: `false`. - -**Examples** - -Query - -```bash -./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" -./clickhouse local -q "select * from test_tbl" -``` - -Result - -```text -a 0 1971-01-01 -``` - ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a4cb0c2dbd9..730b6ab80ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -874,7 +874,6 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6e3e086859b..8eacc7acc97 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,7 +73,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e321e5264ca..af90e4462dd 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -152,7 +152,6 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 244b906549e..79ce2549b4d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -316,54 +315,17 @@ bool CSVFormatReader::readField( return false; } - if (format_settings.csv.use_default_on_bad_values) - return readFieldOrDefault(column, type, serialization); - return readFieldImpl(*buf, column, type, serialization); -} - -bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) -{ if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); } /// Read the column normally. - serialization->deserializeTextCSV(column, istr, format_settings); + serialization->deserializeTextCSV(column, *buf, format_settings); return true; } -bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) -{ - String field; - readCSVField(field, *buf, format_settings.csv); - ReadBufferFromString tmp_buf(field); - bool is_bad_value = false; - bool res = false; - - size_t col_size = column.size(); - try - { - res = readFieldImpl(tmp_buf, column, type, serialization); - /// Check if we parsed the whole field successfully. - if (!field.empty() && !tmp_buf.eof()) - is_bad_value = true; - } - catch (const Exception &) - { - is_bad_value = true; - } - - if (!is_bad_value) - return res; - - if (column.size() == col_size + 1) - column.popBack(1); - column.insertDefault(); - return false; -} - void CSVFormatReader::skipPrefixBeforeHeader() { for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 7b1a1fc433d..8ccf04feed3 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,8 +89,6 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } - bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); - bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference deleted file mode 100644 index 6abcc56bacc..00000000000 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ /dev/null @@ -1,5 +0,0 @@ -0 111 1970-01-01 false -1 abc 2023-03-14 true -2 c 1970-01-01 false -4 888 2023-03-14 false -5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh deleted file mode 100755 index 30223329eca..00000000000 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -# NOTE: this sh wrapper is required because of shell_config - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" -cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" -$CLICKHOUSE_CLIENT -q "select * from test_tbl" -$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv deleted file mode 100644 index e829cc0106a..00000000000 --- a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv +++ /dev/null @@ -1,5 +0,0 @@ -1,abc,2023-03-14,true -2,c,ab,false -bc,111,ab,ban -4,888,2023-03-14,false -5,bks,2023-03,abdd \ No newline at end of file From f0026af1893772a7a14f21d11cc88307ba07500a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:51:11 +0200 Subject: [PATCH 405/473] Revert "Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed"" --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 22 ++++++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/CSVRowInputFormat.cpp | 42 ++++++++++++++++++- .../Formats/Impl/CSVRowInputFormat.h | 2 + ...11_csv_input_field_type_mismatch.reference | 5 +++ .../02811_csv_input_field_type_mismatch.sh | 13 ++++++ .../data_csv/csv_with_bad_field_values.csv | 5 +++ 10 files changed, 91 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference create mode 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed2f010a632..ddf4ab3f78e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,6 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..c8adc83d3ad 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -989,6 +989,28 @@ Result a b ``` +### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} + +Allow to set default value to column when CSV field deserialization failed on bad value + +Default value: `false`. + +**Examples** + +Query + +```bash +./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +./clickhouse local -q "select * from test_tbl" +``` + +Result + +```text +a 0 1971-01-01 +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 730b6ab80ed..a4cb0c2dbd9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -874,6 +874,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8eacc7acc97..6e3e086859b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,6 +73,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; + format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index af90e4462dd..e321e5264ca 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -152,6 +152,7 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; + bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 79ce2549b4d..244b906549e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -315,17 +316,54 @@ bool CSVFormatReader::readField( return false; } + if (format_settings.csv.use_default_on_bad_values) + return readFieldOrDefault(column, type, serialization); + return readFieldImpl(*buf, column, type, serialization); +} + +bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); } /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); + serialization->deserializeTextCSV(column, istr, format_settings); return true; } +bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + String field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromString tmp_buf(field); + bool is_bad_value = false; + bool res = false; + + size_t col_size = column.size(); + try + { + res = readFieldImpl(tmp_buf, column, type, serialization); + /// Check if we parsed the whole field successfully. + if (!field.empty() && !tmp_buf.eof()) + is_bad_value = true; + } + catch (const Exception &) + { + is_bad_value = true; + } + + if (!is_bad_value) + return res; + + if (column.size() == col_size + 1) + column.popBack(1); + column.insertDefault(); + return false; +} + void CSVFormatReader::skipPrefixBeforeHeader() { for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 8ccf04feed3..7b1a1fc433d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } + bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); + bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference new file mode 100644 index 00000000000..6abcc56bacc --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -0,0 +1,5 @@ +0 111 1970-01-01 false +1 abc 2023-03-14 true +2 c 1970-01-01 false +4 888 2023-03-14 false +5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh new file mode 100755 index 00000000000..30223329eca --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" +cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "select * from test_tbl" +$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv new file mode 100644 index 00000000000..e829cc0106a --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -0,0 +1,5 @@ +1,abc,2023-03-14,true +2,c,ab,false +bc,111,ab,ban +4,888,2023-03-14,false +5,bks,2023-03,abdd \ No newline at end of file From bdaf82922a599ab5abb123b4bbd1f41249543ec7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 12:54:25 +0000 Subject: [PATCH 406/473] Use formatAST() --- src/Interpreters/Cache/QueryCache.cpp | 9 ++++----- src/Parsers/formatAST.cpp | 4 ++-- src/Parsers/formatAST.h | 2 +- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 1d1543844a2..a6c509e8bb1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -115,12 +116,10 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } -String queryStringFromAst(ASTPtr ast) +String queryStringFromAST(ASTPtr ast) { WriteBufferFromOwnString buf; - IAST::FormatSettings format_settings(buf, /*one_line*/ true); - format_settings.show_secrets = false; - ast->format(format_settings); + formatAST(*ast, buf, /*hilite*/ false, /*one_line*/ true, /*show_secrets*/ false); return buf.str(); } @@ -138,7 +137,7 @@ QueryCache::Key::Key( , is_shared(is_shared_) , expires_at(expires_at_) , is_compressed(is_compressed_) - , query_string(queryStringFromAst(ast_)) + , query_string(queryStringFromAST(ast_)) { } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index fca8ea0aa35..aa1afe17c75 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -4,9 +4,9 @@ namespace DB { -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line) +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line); + IAST::FormatSettings settings(buf, one_line, show_secrets); settings.hilite = hilite; ast.format(settings); diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index 28af2400a4c..ebd284fc18a 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -11,7 +11,7 @@ class WriteBuffer; /** Takes a syntax tree and turns it back into text. * In case of INSERT query, the data will be missing. */ -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false); +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true); String serializeAST(const IAST & ast, bool one_line = true); From 9c4005b33fc74ab4ec3b68ebc877fdda499e8932 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 19 Jul 2023 15:12:47 +0200 Subject: [PATCH 407/473] Add logging about all found workflows for merge_pr.py --- tests/ci/merge_pr.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 14844ed9b25..35b0614b01f 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -246,6 +246,12 @@ def main(): if args.check_running_workflows: workflows = get_workflows_for_head(repo, pr.head.sha) + logging.info( + "The PR #%s has following workflows:\n%s", + pr.number, + "\n".join(f"{wf.html_url}: status is {wf.status}" for wf in workflows), + ) + workflows_in_progress = [wf for wf in workflows if wf.status != "completed"] # At most one workflow in progress is fine. We check that there no # cases like, e.g. PullRequestCI and DocksCheck in progress at once From 53818dde8cef7dd573217fa049d01b233a076ac2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 15:22:25 +0200 Subject: [PATCH 408/473] MergeTree/ReplicatedMergeTree should use server timezone for log entries Otherwise session_timezone/use_client_time_zone will break things Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index cac26c5ac23..4dbccb91620 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -61,7 +61,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP { auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); *out << "format version: 1\n" - << "create time: " << LocalDateTime(create_time) << "\n"; + << "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n"; *out << "commands: "; commands.writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index ac956433eab..9eb8b6ce24c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -48,7 +48,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const format_version = std::max(format_version, FORMAT_WITH_LOG_ENTRY_ID); out << "format version: " << format_version << "\n" - << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" + << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n" << "source replica: " << source_replica << '\n' << "block_id: " << escape << block_id << '\n'; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1bbb246338c..e2c23ecfe85 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -12,7 +12,7 @@ namespace DB void ReplicatedMergeTreeMutationEntry::writeText(WriteBuffer & out) const { out << "format version: 1\n" - << "create time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" + << "create time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n" << "source replica: " << source_replica << "\n" << "block numbers count: " << block_numbers.size() << "\n"; From 7ad399cc6d97aaa356cbbe50f697b563d0a2f995 Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Wed, 19 Jul 2023 14:25:36 +0100 Subject: [PATCH 409/473] Rephrase ALL supported note I interpreted the note as 'ALL' is only supported before 23.4 I think this reordering makes it clearer --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index c3ddee07d0b..61c8a73673f 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should ``` :::note ALL -`ALL` is only applicable to the `RESTORE` command prior to version 23.4 of Clickhouse. +Prior to version 23.4 of Clickhouse, `ALL` was only applicable to the `RESTORE` command. ::: ## Background From c47f19303afc3e4f18ceddbfe7b8f7e7cc4622b3 Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Wed, 19 Jul 2023 14:27:21 +0100 Subject: [PATCH 410/473] Correct CH capitalisation --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 61c8a73673f..62f931a76b4 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should ``` :::note ALL -Prior to version 23.4 of Clickhouse, `ALL` was only applicable to the `RESTORE` command. +Prior to version 23.4 of ClickHouse, `ALL` was only applicable to the `RESTORE` command. ::: ## Background From 688b55b6ff80ee333ab9ef318d42937d5b5d3064 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 13:29:07 +0000 Subject: [PATCH 411/473] Try to fix test, rename arg --- src/Common/SystemLogBase.cpp | 9 +++++---- src/Common/SystemLogBase.h | 4 ++-- src/Loggers/Loggers.cpp | 6 +++--- tests/queries/0_stateless/02813_starting_in_text_log.sql | 2 +- 4 files changed, 11 insertions(+), 10 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02813_starting_in_text_log.sql diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index bed6d661db7..8cf8103e1c7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -41,9 +41,9 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue( - const String & name_, + const String & table_name_, size_t flush_interval_milliseconds_) - : log(&Poco::Logger::get(name_)) + : log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")")) , flush_interval_milliseconds(flush_interval_milliseconds_) {} @@ -120,6 +120,7 @@ void SystemLogQueue::push(const LogElement & element) template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { + uint64_t this_thread_requested_offset; { @@ -204,10 +205,10 @@ void SystemLogQueue::shutdown() template SystemLogBase::SystemLogBase( - const String& name, + const String& table_name_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : queue(queue_ ? queue_ : std::make_shared>(name, flush_interval_milliseconds_)) + : queue(queue_ ? queue_ : std::make_shared>(table_name_, flush_interval_milliseconds_)) { } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0ac376769ad..3716584be24 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -80,7 +80,7 @@ class SystemLogQueue public: SystemLogQueue( - const String & name_, + const String & table_name_, size_t flush_interval_milliseconds_); void shutdown(); @@ -130,7 +130,7 @@ public: using Self = SystemLogBase; SystemLogBase( - const String& name, + const String& table_name_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 4cc74902ee1..85a8152602f 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -255,10 +255,10 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log #ifndef WITHOUT_TEXT_LOG if (config.has("text_log")) { - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + String text_log_level_str = config.getString("text_log.level", "trace"); + int text_log_level = Poco::Logger::parseLevel(text_log_level_str); size_t flush_interval_milliseconds = config.getUInt64("text_log.flush_interval_milliseconds", - DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); + DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); split->addTextLog(DB::TextLog::getLogQueue(flush_interval_milliseconds), text_log_level); } #endif diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql old mode 100755 new mode 100644 index 8ef78945a72..e007f58189e --- a/tests/queries/0_stateless/02813_starting_in_text_log.sql +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -1,2 +1,2 @@ SYSTEM FLUSH LOGS; -SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Application: Starting ClickHouse%'; +SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Starting ClickHouse%'; From 9f7e40e8e57cc5e8c997dff16b5c6645283ffcb3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 13:43:22 +0000 Subject: [PATCH 412/473] Remove empty line --- src/Common/SystemLogBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 8cf8103e1c7..294ba09e375 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -120,7 +120,6 @@ void SystemLogQueue::push(const LogElement & element) template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { - uint64_t this_thread_requested_offset; { From 777026e42e00311c4a751e8899fac407d6c8b874 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 19 Jul 2023 13:43:40 +0000 Subject: [PATCH 413/473] Fix test --- src/Functions/s2RectAdd.cpp | 8 ++++---- src/Functions/s2RectContains.cpp | 8 ++++---- src/Functions/s2RectIntersection.cpp | 4 ++-- src/Functions/s2RectUnion.cpp | 6 +++--- tests/queries/0_stateless/01849_geoToS2.reference | 1 - tests/queries/0_stateless/01849_geoToS2.sql | 2 +- .../0_stateless/02224_s2_test_const_columns.reference | 2 +- tests/queries/0_stateless/02224_s2_test_const_columns.sql | 2 +- 8 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index 9266f4ae1a7..0be304234cb 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -118,14 +118,14 @@ public: if (!point.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive " "and the longitude is between -180 and 180 degrees inclusive."); if (!rect.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); rect.AddPoint(point.ToPoint()); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index aed123ce8ee..898e12a6466 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -111,14 +111,14 @@ public: if (!point.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive " "and the longitude is between -180 and 180 degrees inclusive."); if (!rect.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); dst_data.emplace_back(rect.Contains(point.ToLatLng())); } diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index ffe26d171d0..f0cc02de9d9 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -133,8 +133,8 @@ public: if (!rect1.is_valid() || !rect2.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " "Also, if either the latitude or longitude bound is empty then both must be."); S2LatLngRect rect_intersection = rect1.Intersection(rect2); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index 472b30c2d55..a5cedd35812 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -131,9 +131,9 @@ public: if (!rect1.is_valid() || !rect2.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); S2LatLngRect rect_union = rect1.Union(rect2); diff --git a/tests/queries/0_stateless/01849_geoToS2.reference b/tests/queries/0_stateless/01849_geoToS2.reference index 08d76978791..a8196994361 100644 --- a/tests/queries/0_stateless/01849_geoToS2.reference +++ b/tests/queries/0_stateless/01849_geoToS2.reference @@ -39,4 +39,3 @@ Checking s2 index generation. (74.0061,-68.32124) (74.0061,-68.32124) ok (10.61077,-64.1841) (10.61077,-64.1841) ok (-89.81096,-57.01398) (-89.81096,-57.01398) ok -4864204703484167331 diff --git a/tests/queries/0_stateless/01849_geoToS2.sql b/tests/queries/0_stateless/01849_geoToS2.sql index abd084a2b19..e997fec14e5 100644 --- a/tests/queries/0_stateless/01849_geoToS2.sql +++ b/tests/queries/0_stateless/01849_geoToS2.sql @@ -44,7 +44,7 @@ SELECT first, second, result FROM ( SELECT s2ToGeo(toUInt64(-1)); -- { serverError 36 } SELECT s2ToGeo(nan); -- { serverError 43 } -SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); +SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } SELECT geoToS2(nan, nan); -- { serverError 43 } SELECT geoToS2(-inf, 1.1754943508222875e-38); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02224_s2_test_const_columns.reference b/tests/queries/0_stateless/02224_s2_test_const_columns.reference index 9982596f097..20f32ec6be7 100644 --- a/tests/queries/0_stateless/02224_s2_test_const_columns.reference +++ b/tests/queries/0_stateless/02224_s2_test_const_columns.reference @@ -16,4 +16,4 @@ (5179062030687166815,5177056748191934217) (5179062030687166815,5177057445452335297) (5178914411069187297,5177056748191934217) -(6304347505408739331,8070450532247928833) +(5178914411069187297,5177912432982045463) diff --git a/tests/queries/0_stateless/02224_s2_test_const_columns.sql b/tests/queries/0_stateless/02224_s2_test_const_columns.sql index f33a7f2b696..1d3e51065b5 100644 --- a/tests/queries/0_stateless/02224_s2_test_const_columns.sql +++ b/tests/queries/0_stateless/02224_s2_test_const_columns.sql @@ -9,4 +9,4 @@ SELECT s2CapUnion(3814912406305146967, toFloat64(1), 1157347770437378819, toFloa SELECT s2RectAdd(5178914411069187297, 5177056748191934217, arrayJoin([5179056748191934217,5177914411069187297])); SELECT s2RectContains(5179062030687166815, 5177056748191934217, arrayJoin([5177914411069187297, 5177914411069187297])); SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217, 5177914411069187297])); -SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217,1157347770437378819])); +SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217,5177914411069187297])); From 90be5e6160f2342121dfe2d014f4d247a4efa39c Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Wed, 19 Jul 2023 20:04:43 +0100 Subject: [PATCH 414/473] Remove reference to `TIMEOUT` in live views documentation Temporary live views were removed in #42173, but the documentation was not fully updated to reflect the change. --- docs/en/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 10b15638152..11026340a0f 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -97,7 +97,7 @@ This is an experimental feature that may change in backwards-incompatible ways i ::: ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` Live views store result of the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query and are updated any time the result of the query changes. Query result as well as partial result needed to combine with new data are stored in memory providing increased performance for repeated queries. Live views can provide push notifications when query result changes using the [WATCH](../../../sql-reference/statements/watch.md) query. From 02fe735b768e2d171191091c402f4732ace4669e Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Wed, 19 Jul 2023 20:12:35 +0100 Subject: [PATCH 415/473] fix in other other languages --- docs/ru/sql-reference/statements/create/view.md | 2 +- docs/zh/sql-reference/statements/create/view.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index d3846aac289..1a60dc0716c 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -73,7 +73,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na Чтобы использовать `LIVE VIEW` и запросы `WATCH`, включите настройку [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view). ::: ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` `LIVE VIEW` хранит результат запроса [SELECT](../../../sql-reference/statements/select/index.md), указанного при создании, и обновляется сразу же при изменении этого результата. Конечный результат запроса и промежуточные данные, из которых формируется результат, хранятся в оперативной памяти, и это обеспечивает высокую скорость обработки для повторяющихся запросов. LIVE-представления могут отправлять push-уведомления при изменении результата исходного запроса `SELECT`. Для этого используйте запрос [WATCH](../../../sql-reference/statements/watch.md). diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index 8ce2d20a10c..bce0994ecd2 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -72,7 +72,7 @@ ClickHouse 中的物化视图更像是插入触发器。 如果视图查询中 使用[allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view)设置启用实时视图和`WATCH`查询的使用。 输入命令`set allow_experimental_live_view = 1`。 ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` 实时视图存储相应[SELECT](../../../sql-reference/statements/select/index.md)查询的结果,并在查询结果更改时随时更新。 查询结果以及与新数据结合所需的部分结果存储在内存中,为重复查询提供更高的性能。当使用[WATCH](../../../sql-reference/statements/watch.md)查询更改查询结果时,实时视图可以提供推送通知。 From e7b8767585e748d91796e669b871d40546c40bc8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 22:14:05 +0200 Subject: [PATCH 416/473] Mark test 02125_many_mutations_2 as no-parallel to avoid flakiness --- tests/queries/0_stateless/02125_many_mutations_2.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index 5b779c1b276..819ac8c9524 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From ff235e0f3078f6c27a9a1ab1383a91378313ab77 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 20 Jul 2023 05:41:39 +0000 Subject: [PATCH 417/473] Turn off log in queue, fix data race --- src/Common/SystemLogBase.cpp | 9 +++++++-- src/Common/SystemLogBase.h | 3 ++- src/Interpreters/TextLog.h | 6 ++++-- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 294ba09e375..d1845a292b9 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -42,10 +42,14 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue( const String & table_name_, - size_t flush_interval_milliseconds_) + size_t flush_interval_milliseconds_, + bool turn_off_logger_) : log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")")) , flush_interval_milliseconds(flush_interval_milliseconds_) -{} +{ + if (turn_off_logger_) + log->setLevel(0); +} static thread_local bool recursive_push_call = false; @@ -197,6 +201,7 @@ SystemLogQueue::Index SystemLogQueue::pop(std::vector void SystemLogQueue::shutdown() { + std::unique_lock lock(mutex); is_shutdown = true; /// Tell thread to shutdown. flush_event.notify_all(); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 3716584be24..f6e4a579edf 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -81,7 +81,8 @@ class SystemLogQueue public: SystemLogQueue( const String & table_name_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + bool turn_off_logger_ = false); void shutdown(); diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 0febce03abc..60ca11632aa 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -40,6 +40,8 @@ struct TextLogElement class TextLog : public SystemLog { public: + using Queue = SystemLogQueue; + TextLog( ContextPtr context_, const String & database_name_, @@ -47,9 +49,9 @@ public: const String & storage_def_, size_t flush_interval_milliseconds_); - static std::shared_ptr> getLogQueue(size_t flush_interval_milliseconds) + static std::shared_ptr getLogQueue(size_t flush_interval_milliseconds) { - static std::shared_ptr> queue = std::make_shared>("text_log", flush_interval_milliseconds); + static std::shared_ptr queue = std::make_shared("text_log", flush_interval_milliseconds, true); return queue; } }; From 27ca367b2cb349391946c45d7e3d22fe6d543c42 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jul 2023 10:01:07 +0200 Subject: [PATCH 418/473] ZooKeeperRetriesControl rethrows with original callstack. --- src/Storages/MergeTree/ZooKeeperRetries.h | 44 ++++++++++++++++++----- 1 file changed, 35 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index e55b04c27b3..512c0800de7 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -72,7 +72,7 @@ public: if (!Coordination::isHardwareError(e.code)) throw; - setKeeperError(e.code, e.message()); + setKeeperError(std::current_exception(), e.code, e.message()); } catch (...) { @@ -91,16 +91,16 @@ public: } catch (const zkutil::KeeperException & e) { - setKeeperError(e.code, e.message()); + setKeeperError(std::current_exception(), e.code, e.message()); } catch (const Exception & e) { - setUserError(e.code(), e.what()); + setUserError(std::current_exception(), e.code(), e.what()); } return false; } - void setUserError(int code, std::string message) + void setUserError(std::exception_ptr exception, int code, std::string message) { if (retries_info.logger) LOG_TRACE( @@ -113,16 +113,28 @@ public: iteration_succeeded = false; user_error.code = code; user_error.message = std::move(message); + user_error.exception = exception; keeper_error = KeeperError{}; } + template + void setUserError(std::exception_ptr exception, int code, fmt::format_string fmt, Args &&... args) + { + setUserError(exception, code, fmt::format(fmt, std::forward(args)...)); + } + + void setUserError(int code, std::string message) + { + setUserError(std::make_exception_ptr(Exception::createDeprecated(message, code)), code, message); + } + template void setUserError(int code, fmt::format_string fmt, Args &&... args) { setUserError(code, fmt::format(fmt, std::forward(args)...)); } - void setKeeperError(Coordination::Error code, std::string message) + void setKeeperError(std::exception_ptr exception, Coordination::Error code, std::string message) { if (retries_info.logger) LOG_TRACE( @@ -135,9 +147,21 @@ public: iteration_succeeded = false; keeper_error.code = code; keeper_error.message = std::move(message); + keeper_error.exception = exception; user_error = UserError{}; } + template + void setKeeperError(std::exception_ptr exception, Coordination::Error code, fmt::format_string fmt, Args &&... args) + { + setKeeperError(exception, code, fmt::format(fmt, std::forward(args)...)); + } + + void setKeeperError(Coordination::Error code, std::string message) + { + setKeeperError(std::make_exception_ptr(zkutil::KeeperException(message, code)), code, message); + } + template void setKeeperError(Coordination::Error code, fmt::format_string fmt, Args &&... args) { @@ -163,12 +187,14 @@ private: using Code = Coordination::Error; Code code = Code::ZOK; std::string message; + std::exception_ptr exception; }; struct UserError { int code = ErrorCodes::OK; std::string message; + std::exception_ptr exception; }; bool canTry() @@ -232,11 +258,11 @@ private: void throwIfError() const { - if (user_error.code != ErrorCodes::OK) - throw Exception::createDeprecated(user_error.message, user_error.code); + if (user_error.exception) + std::rethrow_exception(user_error.exception); - if (keeper_error.code != KeeperError::Code::ZOK) - throw zkutil::KeeperException(keeper_error.message, keeper_error.code); + if (keeper_error.exception) + std::rethrow_exception(keeper_error.exception); } void logLastError(std::string_view header) From c7ab6e908adf2a088ad41e00ea2bfad5ea16526a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 20 Jul 2023 08:55:22 +0000 Subject: [PATCH 419/473] Move tode to to try to make the diff simpler --- src/Common/SystemLogBase.cpp | 60 ++++++++++++++++++------------------ 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index d1845a292b9..ed5ffd78a7b 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -121,6 +121,36 @@ void SystemLogQueue::push(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) +{ + // Use an arbitrary timeout to avoid endless waiting. 60s proved to be + // too fast for our parallel functional tests, probably because they + // heavily load the disk. + const int timeout_seconds = 180; + std::unique_lock lock(mutex); + bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + { + return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; + }); + + if (!result) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", + toString(timeout_seconds), demangle(typeid(*this).name())); + } +} + template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { @@ -145,26 +175,6 @@ uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyw return this_thread_requested_offset; } -template -void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) -{ - // Use an arbitrary timeout to avoid endless waiting. 60s proved to be - // too fast for our parallel functional tests, probably because they - // heavily load the disk. - const int timeout_seconds = 180; - std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] - { - return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; - }); - - if (!result) - { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", - toString(timeout_seconds), demangle(typeid(*this).name())); - } -} - template void SystemLogQueue::confirm(uint64_t to_flush_end) { @@ -229,16 +239,6 @@ void SystemLogBase::add(const LogElement & element) queue->push(element); } -template -void SystemLogBase::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - template void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } From 067e3caa2c43ed981a7c598c45668f37b0ac32c6 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 18:13:19 +0800 Subject: [PATCH 420/473] Remove constants from description_sorted_. --- src/Processors/Transforms/FinishSortingTransform.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 066928446f2..744d035d0ee 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -38,15 +38,11 @@ FinishSortingTransform::FinishSortingTransform( /// Remove constants from description_sorted_. SortDescription description_sorted_without_constants; description_sorted_without_constants.reserve(description_sorted_.size()); - size_t num_columns = header.columns(); - ColumnNumbers map(num_columns, num_columns); for (const auto & column_description : description_sorted_) { - auto old_pos = header.getPositionByName(column_description.column_name); - auto new_pos = map[old_pos]; + auto pos = header.getPositionByName(column_description.column_name); - if (new_pos < num_columns) - { + if (!const_columns_to_remove[pos]){ description_sorted_without_constants.push_back(column_description); } } From 0ba97eeea597ad027c375cf292419dd555a9cb73 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 15 Jun 2023 08:05:47 +0800 Subject: [PATCH 421/473] wip: grace hash join support full & right join --- docs/en/operations/settings/settings.md | 2 + src/Interpreters/GraceHashJoin.cpp | 31 +++++++-- src/Interpreters/GraceHashJoin.h | 3 +- .../Transforms/JoiningTransform.cpp | 65 +++++++++++++++++-- src/Processors/Transforms/JoiningTransform.h | 24 ++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...01721_join_implicit_cast_long.reference.j2 | 40 ------------ .../01721_join_implicit_cast_long.sql.j2 | 1 - .../02273_full_sort_join.reference.j2 | 18 +---- .../0_stateless/02273_full_sort_join.sql.j2 | 4 +- ...274_full_sort_join_nodistinct.reference.j2 | 34 +--------- .../02274_full_sort_join_nodistinct.sql.j2 | 6 +- .../02275_full_sort_join_long.reference | 24 ++++++- .../02275_full_sort_join_long.sql.j2 | 9 +-- 14 files changed, 138 insertions(+), 125 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 22aeecf4335..580b51a984d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -537,6 +537,8 @@ Possible values: The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. + - hash [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index edf604bc0b4..f94453293f6 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -301,8 +301,10 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { + bool is_asof = (table_join->strictness() == JoinStrictness::Asof); - return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct(); + auto kind = table_join->kind(); + return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); } GraceHashJoin::~GraceHashJoin() = default; @@ -322,7 +324,6 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con /// One row can't be split, avoid loop if (total_rows < 2) return false; - bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes); if (has_overflow) @@ -494,17 +495,30 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const +/// Each bucket are handled by the following steps +/// 1. build hash_join by the right side blocks. +/// 2. join left side with the hash_join, +/// 3. read right non-joined blocks from hash_join. +/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted. +/// +/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +/// only one processor could take the non-joined blocks from right stream, and ensure all rows from +/// left stream have been emitted before this. +IBlocksStreamPtr +GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - /// We do no support returning non joined blocks here. - /// TODO: They _should_ be reported by getDelayedBlocks instead - return nullptr; + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream { public: - explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_) + explicit DelayedBlocks( + size_t current_bucket_, + Buckets buckets_, + InMemoryJoinPtr hash_join_, + const Names & left_key_names_, + const Names & right_key_names_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) @@ -522,12 +536,15 @@ public: do { + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform. + // There is a lock inside left_reader.read() . block = left_reader.read(); if (!block) { return {}; } + // block comes from left_reader, need to join with right table to get the result. Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); block = std::move(blocks[current_idx]); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bce04ee6b04..ce519892b0e 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -13,7 +13,6 @@ namespace DB { - class TableJoin; class HashJoin; @@ -79,7 +78,7 @@ public: bool supportTotals() const override { return false; } IBlocksStreamPtr - getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override; /// Open iterator over joined blocks. /// Must be called after all @joinBlock calls. diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 49b90d04b81..f1ceefbf229 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -189,7 +189,6 @@ void JoiningTransform::transform(Chunk & chunk) } else block = readExecute(chunk); - auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } @@ -311,8 +310,16 @@ void FillingRightJoinSideTransform::work() } -DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header) - : IProcessor(InputPorts{Block()}, OutputPorts{output_header}) +DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_) + : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) + , left_header(left_header_) + , output_header(output_header_) + , max_block_size(max_block_size_) + , join(join_) { } @@ -365,6 +372,7 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else @@ -387,11 +395,24 @@ void DelayedJoinedBlocksWorkerTransform::work() if (!task) return; - Block block = task->delayed_blocks->next(); + Block block; + if (!left_delayed_stream_finished) + { + block = task->delayed_blocks->next(); + if (!block) + { + left_delayed_stream_finished = true; + block = nextNonJoinedBlock(); + } + } + else + { + block = nextNonJoinedBlock(); + } if (!block) { - task.reset(); + resetTask(); return; } @@ -400,6 +421,31 @@ void DelayedJoinedBlocksWorkerTransform::work() output_chunk.setColumns(block.getColumns(), rows); } +void DelayedJoinedBlocksWorkerTransform::resetTask() +{ + task.reset(); + left_delayed_stream_finished = false; + non_joined_delayed_stream = nullptr; +} + +Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() +{ + // Before read from non-joined stream, all blocks in left file reader must have been joined. + // For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before + // the all blocks in left file reader have been finished, since the used flags are incomplete. + // To make only one processor could read from non-joined stream seems be a easy way. + if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast()) + { + non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + } + + if (non_joined_delayed_stream) + { + return non_joined_delayed_stream->next(); + } + return {}; +} + DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_) : IProcessor(InputPorts{}, OutputPorts(num_streams, Block())) , join(std::move(join_)) @@ -433,6 +479,9 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (finished) { + // Since have memory limit, cannot handle all buckets parallelly by different + // DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs. + // Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket. for (auto & output : outputs) { if (output.isFinished()) @@ -448,10 +497,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { + // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform + // could read right non-joined blocks from the join. + auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); + auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); + chunk.setChunkInfo(task); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..10b413ed4e5 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -116,9 +116,14 @@ class DelayedBlocksTask : public ChunkInfo public: explicit DelayedBlocksTask() : finished(true) {} - explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {} + explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) + : delayed_blocks(std::move(delayed_blocks_)) + , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) + { + } IBlocksStreamPtr delayed_blocks = nullptr; + JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; bool finished = false; }; @@ -147,7 +152,11 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: - explicit DelayedJoinedBlocksWorkerTransform(Block output_header); + explicit DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -155,10 +164,19 @@ public: void work() override; private: + Block left_header; + Block output_header; + size_t max_block_size; + JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - bool finished = false; + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left + bool left_delayed_stream_finished = false; + IBlocksStreamPtr non_joined_delayed_stream = nullptr; + + void resetTask(); + Block nextNonJoinedBlock(); }; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index dedf85e409c..ba98d725532 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(joined_header); + auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index e9f32087439..ae43aa7195c 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,7 +1,6 @@ {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -17,7 +16,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -30,7 +28,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -41,7 +38,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -49,7 +45,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -65,7 +60,6 @@ 8 8 0 9 9 0 10 10 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -78,7 +72,6 @@ 9 9 0 10 10 0 = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -89,7 +82,6 @@ 3 3 3 4 4 4 5 5 5 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -98,7 +90,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -114,7 +105,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -127,7 +117,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -138,7 +127,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -146,7 +134,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -162,7 +149,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -175,7 +161,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -186,7 +171,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -196,7 +180,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -205,13 +188,11 @@ 1 55 1055 0 0 -10 0 990 1 55 15 1055 1015 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -219,11 +200,9 @@ 1 1 1 -{% endif -%} {% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -239,7 +218,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -252,7 +230,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -263,7 +240,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -271,7 +247,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -287,7 +262,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -300,7 +274,6 @@ 9 9 \N 10 10 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -311,7 +284,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -320,7 +292,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -336,7 +307,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -349,7 +319,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -360,7 +329,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -368,7 +336,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -384,7 +351,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -397,7 +363,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -408,7 +373,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -418,7 +382,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -427,13 +390,11 @@ 1 55 1055 1 55 15 1055 1015 \N \N -10 \N 990 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -442,5 +403,4 @@ 1 1 {% endif -%} -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index f5321939f28..38f71f4c5ec 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -10,7 +10,6 @@ INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1 INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 98bfd9d9b2b..0af4158e971 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,7 +1,7 @@ {% set table_size = 15 -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} -- {{ join_algorithm }} -- -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} ALL INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -50,7 +50,6 @@ ALL LEFT | bs = {{ block_size }} 14 14 val9 0 14 14 val9 0 ALL RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -64,7 +63,6 @@ ALL RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -85,7 +83,6 @@ ALL LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -99,7 +96,6 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 -{% endif -%} ANY INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -137,7 +133,6 @@ ANY LEFT | bs = {{ block_size }} 13 13 val13 0 14 14 val9 0 ANY RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -150,7 +145,6 @@ ANY RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | bs = {{ block_size }} | copmosite key @@ -170,7 +164,6 @@ ANY LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -183,7 +176,6 @@ ANY RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 1 \N 1 val3 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 -{% endif -%} {% endfor -%} ALL INNER | join_use_nulls = 1 4 4 0 0 @@ -219,7 +211,6 @@ ALL LEFT | join_use_nulls = 1 14 14 val9 0 14 14 val9 0 ALL RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -233,7 +224,6 @@ ALL RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -254,7 +244,6 @@ ALL LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 @@ -268,7 +257,6 @@ ALL RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} ANY INNER | join_use_nulls = 1 4 4 0 0 5 5 0 0 @@ -296,7 +284,6 @@ ANY LEFT | join_use_nulls = 1 13 13 val13 0 14 14 val9 0 ANY RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -309,7 +296,6 @@ ANY RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | join_use_nulls = 1 | copmosite key @@ -329,7 +315,6 @@ ANY LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 \N \N \N 1 1 1 \N val7 @@ -342,5 +327,4 @@ ANY RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 43f7354017c..6b6aa53836e 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -28,9 +28,7 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); - {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +38,7 @@ SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} {% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2cc6c6e85d6..df968e86e8d 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -1,6 +1,6 @@ {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} --- {{ join_algorithm }} --- -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} t1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -108,7 +108,6 @@ t1 ALL LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -161,7 +160,6 @@ t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val28 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 2 2 5 5 @@ -177,7 +175,6 @@ t1 ANY LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -188,9 +185,7 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 2 2 5 5 @@ -243,9 +238,7 @@ t1 ALL FULL JOIN t2 | bs = {{ block_size }} 2 2 5 5 2 2 5 5 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 5 @@ -298,7 +291,6 @@ t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} 2 5 5 2 5 5 3 4 4 -{% endif -%} t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -315,7 +307,6 @@ t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -326,7 +317,6 @@ t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -341,7 +331,6 @@ t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -352,9 +341,7 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 5 0 \N 0 5 0 \N 0 5 @@ -372,9 +359,8 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 -{% endif -%} -t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} {% if join_algorithm != 'grace_hash' -%} +t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 2 5 0 @@ -409,7 +395,6 @@ tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -420,7 +405,6 @@ tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -435,7 +419,6 @@ tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -446,9 +429,7 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -466,9 +447,7 @@ tn1 ALL FULL JOIN t2 | bs = {{ block_size }} \N 2 0 5 \N 2 0 5 \N 2 0 5 -{% endif -%} tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 0 5 @@ -486,7 +465,6 @@ tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -503,7 +481,6 @@ tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -514,7 +491,6 @@ tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -529,7 +505,6 @@ tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -540,9 +515,7 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -560,9 +533,8 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 -{% endif -%} -tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} {% if join_algorithm != 'grace_hash' -%} +tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 3 4 4 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 613da65421e..f8eb4b1a53e 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -16,7 +16,6 @@ INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2 INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -27,7 +26,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SELECT '--- {{ join_algorithm }} ---'; -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} @@ -47,9 +46,10 @@ SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT J SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} +{% if join_algorithm == 'full_sorting_merge' or t2 != 'tn2' -%} SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} - +{% endif -%} {% endfor -%} {% endfor -%} SET max_bytes_in_join = 0; diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index 9ec06aea3e6..73482358d12 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -41,16 +41,34 @@ ALL INNER ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 7276e77dc16..621352f9c25 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,11 +22,6 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; -{% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } -SELECT 'skipped'; -{% endif -%} -{% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +35,6 @@ SET join_algorithm = '{{ join_algorithm }}'; SET max_block_size = {{ block_size }}; -{% if not (kind == 'ANY' and join_algorithm == 'grace_hash') -%} SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 @@ -58,9 +52,8 @@ SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 {{ kind }} RIGHT JOIN t2 ON t1.key == t2.key -; {{ is_implemented(join_algorithm) }} +; -{% endif -%} {% endfor -%} {% endfor -%} From 91dc6a35e17417a44de46d76c0f0214911615244 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 19 Jul 2023 09:18:16 +0800 Subject: [PATCH 422/473] update --- src/Interpreters/GraceHashJoin.cpp | 1 - .../Transforms/JoiningTransform.cpp | 19 +++++-------------- src/Processors/Transforms/JoiningTransform.h | 14 ++++---------- src/QueryPipeline/QueryPipelineBuilder.cpp | 5 ++++- 4 files changed, 13 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index f94453293f6..5d72cf20740 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -301,7 +301,6 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { - bool is_asof = (table_join->strictness() == JoinStrictness::Asof); auto kind = table_join->kind(); return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index f1ceefbf229..5480fea27a4 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -311,15 +311,10 @@ void FillingRightJoinSideTransform::work() DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( - Block left_header_, Block output_header_, - size_t max_block_size_, - JoinPtr join_) + NonJoinedStreamBuilder non_joined_stream_builder_) : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) - , left_header(left_header_) - , output_header(output_header_) - , max_block_size(max_block_size_) - , join(join_) + , non_joined_stream_builder(std::move(non_joined_stream_builder_)) { } @@ -396,15 +391,12 @@ void DelayedJoinedBlocksWorkerTransform::work() return; Block block; - if (!left_delayed_stream_finished) + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left + if (!task->delayed_blocks->isFinished()) { block = task->delayed_blocks->next(); - if (!block) - { - left_delayed_stream_finished = true; block = nextNonJoinedBlock(); - } } else { @@ -424,7 +416,6 @@ void DelayedJoinedBlocksWorkerTransform::work() void DelayedJoinedBlocksWorkerTransform::resetTask() { task.reset(); - left_delayed_stream_finished = false; non_joined_delayed_stream = nullptr; } @@ -436,7 +427,7 @@ Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() // To make only one processor could read from non-joined stream seems be a easy way. if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast()) { - non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + non_joined_delayed_stream = non_joined_stream_builder(); } if (non_joined_delayed_stream) diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 10b413ed4e5..5e7403dbbdb 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -152,11 +152,10 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: + using NonJoinedStreamBuilder = std::function; explicit DelayedJoinedBlocksWorkerTransform( - Block left_header_, Block output_header_, - size_t max_block_size_, - JoinPtr join_); + NonJoinedStreamBuilder non_joined_stream_builder_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -164,15 +163,10 @@ public: void work() override; private: - Block left_header; - Block output_header; - size_t max_block_size; - JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - - /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left - bool left_delayed_stream_finished = false; + /// For building a block stream to access the non-joined rows. + NonJoinedStreamBuilder non_joined_stream_builder; IBlocksStreamPtr non_joined_delayed_stream = nullptr; void resetTask(); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index ba98d725532..553b18dd57b 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,10 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); + auto delayed = std::make_shared( + joined_header, + [left_header, joined_header, max_block_size, join]() + { return join->getNonJoinedBlocks(left_header, joined_header, max_block_size); }); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); From 7a0de384d498497fd026283a8232fcb8ed8ea5e6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 16:46:51 +0000 Subject: [PATCH 423/473] Cosmetics --- src/Parsers/IAST.h | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 7a8ab36518d..f6b7f91fec8 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -196,20 +196,23 @@ public: bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). - - // Newline or whitespace. - char nl_or_ws; + char nl_or_ws; /// Newline or whitespace. FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true) - : ostr(ostr_), one_line(one_line_), show_secrets(show_secrets_) + : ostr(ostr_) + , one_line(one_line_) + , show_secrets(show_secrets_) { nl_or_ws = one_line ? ' ' : '\n'; } FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) - : ostr(ostr_), hilite(other.hilite), one_line(other.one_line), - always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style), - show_secrets(other.show_secrets) + : ostr(ostr_) + , hilite(other.hilite) + , one_line(other.one_line) + , always_quote_identifiers(other.always_quote_identifiers) + , identifier_quoting_style(other.identifier_quoting_style) + , show_secrets(other.show_secrets) { nl_or_ws = one_line ? ' ' : '\n'; } From 35a4fabc2d66ea28b3de3d77df4cfea4b91df870 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:03:04 +0000 Subject: [PATCH 424/473] Make IAST::FormatSettings more regular --- src/Parsers/IAST.cpp | 4 +++- src/Parsers/IAST.h | 3 +-- src/Parsers/formatAST.cpp | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0138372ce89..bf4d6fc9dec 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -170,7 +170,9 @@ size_t IAST::checkDepthImpl(size_t max_depth) const String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const { WriteBufferFromOwnString buf; - format({buf, one_line, show_secrets}); + FormatSettings settings(buf, one_line); + settings.show_secrets = show_secrets; + format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index f6b7f91fec8..13b2e5d9867 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -198,10 +198,9 @@ public: bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. - FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true) + FormatSettings(WriteBuffer & ostr_, bool one_line_) : ostr(ostr_) , one_line(one_line_) - , show_secrets(show_secrets_) { nl_or_ws = one_line ? ' ' : '\n'; } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index aa1afe17c75..bc7faf4bd1d 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -6,9 +6,9 @@ namespace DB void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line, show_secrets); + IAST::FormatSettings settings(buf, one_line); settings.hilite = hilite; - + settings.show_secrets = show_secrets; ast.format(settings); } From e5ec6a1523529db3d1b9d7f137997076c8c2adde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:21:03 +0000 Subject: [PATCH 425/473] Make IAST::FormatSettings more regular, pt. II --- src/Parsers/IAST.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 13b2e5d9867..8e2971d0355 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -191,8 +191,8 @@ public: struct FormatSettings { WriteBuffer & ostr; - bool hilite = false; bool one_line; + bool hilite = false; bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). @@ -207,13 +207,13 @@ public: FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) : ostr(ostr_) - , hilite(other.hilite) , one_line(other.one_line) + , hilite(other.hilite) , always_quote_identifiers(other.always_quote_identifiers) , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) + , nl_or_ws(other.nl_or_ws) { - nl_or_ws = one_line ? ' ' : '\n'; } void writeIdentifier(const String & name) const; From 25ddcc256b04de71e84935cb60a53190c114a494 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:31:53 +0000 Subject: [PATCH 426/473] Make IAST::FormatSettings more regular, pt. III --- src/Parsers/IAST.h | 22 ++++++++++++++----- src/Parsers/formatAST.cpp | 3 +-- src/Parsers/getInsertQuery.cpp | 4 +--- src/Parsers/tests/gtest_format_hiliting.cpp | 3 +-- src/Processors/QueryPlan/ReadFromRemote.cpp | 4 +--- .../MeiliSearch/StorageMeiliSearch.cpp | 7 +++--- src/Storages/StorageDistributed.cpp | 6 ++--- src/Storages/StorageReplicatedMergeTree.cpp | 3 +-- .../transformQueryForExternalDatabase.cpp | 7 +++--- 9 files changed, 31 insertions(+), 28 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 8e2971d0355..d217876459f 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -192,17 +192,27 @@ public: { WriteBuffer & ostr; bool one_line; - bool hilite = false; - bool always_quote_identifiers = false; - IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; - bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). + bool hilite; + bool always_quote_identifiers; + IdentifierQuotingStyle identifier_quoting_style; + bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. - FormatSettings(WriteBuffer & ostr_, bool one_line_) + explicit FormatSettings( + WriteBuffer & ostr_, + bool one_line_, + bool hilite_ = false, + bool always_quote_identifiers_ = false, + IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, + bool show_secrets_ = true) : ostr(ostr_) , one_line(one_line_) + , hilite(hilite_) + , always_quote_identifiers(always_quote_identifiers_) + , identifier_quoting_style(identifier_quoting_style_) + , show_secrets(show_secrets_) + , nl_or_ws(one_line ? ' ' : '\n') { - nl_or_ws = one_line ? ' ' : '\n'; } FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index bc7faf4bd1d..ae2c4a6fcad 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -6,8 +6,7 @@ namespace DB void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line); - settings.hilite = hilite; + IAST::FormatSettings settings(buf, one_line, hilite); settings.show_secrets = show_secrets; ast.format(settings); } diff --git a/src/Parsers/getInsertQuery.cpp b/src/Parsers/getInsertQuery.cpp index 6f52056dfe2..9d111b147bd 100644 --- a/src/Parsers/getInsertQuery.cpp +++ b/src/Parsers/getInsertQuery.cpp @@ -19,9 +19,7 @@ std::string getInsertQuery(const std::string & db_name, const std::string & tabl query.columns->children.emplace_back(std::make_shared(column.name)); WriteBufferFromOwnString buf; - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = quoting; + IAST::FormatSettings settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true, /*identifier_quoting_style*/ quoting); query.IAST::format(settings); return buf.str(); } diff --git a/src/Parsers/tests/gtest_format_hiliting.cpp b/src/Parsers/tests/gtest_format_hiliting.cpp index d0ce8f2c897..a4c3ed86182 100644 --- a/src/Parsers/tests/gtest_format_hiliting.cpp +++ b/src/Parsers/tests/gtest_format_hiliting.cpp @@ -51,8 +51,7 @@ void compare(const String & expected, const String & query) ASTPtr ast = parseQuery(parser, query, 0, 0); WriteBufferFromOwnString write_buffer; - IAST::FormatSettings settings(write_buffer, true); - settings.hilite = true; + IAST::FormatSettings settings(write_buffer, true, true); ast->format(settings); ASSERT_PRED2(HiliteComparator::are_equal_with_hilites_removed, expected, write_buffer.str()); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ed740e3e242..5cc13f45df4 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -86,9 +86,7 @@ static String formattedAST(const ASTPtr & ast) return {}; WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.hilite = false; - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); ast->format(ast_format_settings); return buf.str(); } diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index 5d77fc080a4..aa8b437263a 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -62,9 +62,10 @@ ColumnsDescription StorageMeiliSearch::getTableStructureFromData(const MeiliSear String convertASTtoStr(ASTPtr ptr) { WriteBufferFromOwnString out; - IAST::FormatSettings settings(out, true); - settings.identifier_quoting_style = IdentifierQuotingStyle::BackticksMySQL; - settings.always_quote_identifiers = IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None; + IAST::FormatSettings settings( + out, /*one_line*/ true, /*hilite*/ false, + /*always_quote_identifiers*/ IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None, + /*identifier_quoting_style*/ IdentifierQuotingStyle::BackticksMySQL); ptr->format(settings); return out.str(); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b..e02d7f32b98 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -906,8 +906,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu String new_query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers_=*/ true); new_query->IAST::format(ast_format_settings); new_query_str = buf.str(); } @@ -968,8 +967,7 @@ std::optional StorageDistributed::distributedWriteFromClusterStor String new_query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); new_query->IAST::format(ast_format_settings); new_query_str = buf.str(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5f20c497cb8..52f478d7729 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5074,8 +5074,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu String query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); query.IAST::format(ast_format_settings); query_str = buf.str(); } diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 548b55749d7..375510e62bf 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -334,9 +334,10 @@ String transformQueryForExternalDatabaseImpl( dropAliases(select_ptr); WriteBufferFromOwnString out; - IAST::FormatSettings settings(out, true); - settings.identifier_quoting_style = identifier_quoting_style; - settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + IAST::FormatSettings settings( + out, /*one_line*/ true, /*hilite*/ false, + /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, + /*identifier_quoting_style*/ identifier_quoting_style); select->format(settings); From bd761c365a95e97f1a92638f145353d54a4f2db5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 18:02:09 +0000 Subject: [PATCH 427/473] Make serializeAST() more regular --- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/Interpreters/Cache/QueryCache.h | 2 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Parsers/formatAST.cpp | 4 ++-- src/Parsers/formatAST.h | 9 +++++---- src/Parsers/tests/gtest_Parser.cpp | 10 ++++++++-- src/Parsers/tests/gtest_dictionary_parser.cpp | 10 +++++----- .../Transforms/CheckConstraintsTransform.cpp | 4 ++-- src/Storages/ConstraintsDescription.cpp | 2 +- src/Storages/IndicesDescription.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 2 +- 11 files changed, 28 insertions(+), 21 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 81d5b7372f3..a9a0e972bd1 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -32,7 +32,7 @@ namespace /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function, true); + auto disk_setting_string = serializeAST(function); disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index a67adcc86c9..c24b09c8e46 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -60,7 +60,7 @@ public: /// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the /// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in - /// the SYSTEM.QUERY_CACHE. + /// SYSTEM.QUERY_CACHE. const String query_string; /// Ctor to construct a Key for writing into query cache. diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 5acfe500b1d..398bea26b87 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -518,7 +518,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String static String getCleanQueryAst(const ASTPtr q, ContextPtr context) { - String res = serializeAST(*q, true); + String res = serializeAST(*q); if (auto * masker = SensitiveDataMasker::getInstance()) masker->wipeSensitiveData(res); diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index ae2c4a6fcad..9315279eae6 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -11,10 +11,10 @@ void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, ast.format(settings); } -String serializeAST(const IAST & ast, bool one_line) +String serializeAST(const IAST & ast) { WriteBufferFromOwnString buf; - formatAST(ast, buf, false, one_line); + formatAST(ast, buf, false, true); return buf.str(); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index ebd284fc18a..dd72a59b4a2 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -8,12 +8,13 @@ namespace DB class WriteBuffer; -/** Takes a syntax tree and turns it back into text. - * In case of INSERT query, the data will be missing. - */ +/// Takes a syntax tree and turns it into text. +/// Intended for pretty-printing (multi-line + hiliting). +/// In case of INSERT query, the data will be missing. void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true); -String serializeAST(const IAST & ast, bool one_line = true); +/// Like formatAST() but intended for serialization w/o pretty-printing (single-line, no hiliting). +String serializeAST(const IAST & ast); inline WriteBuffer & operator<<(WriteBuffer & buf, const IAST & ast) { diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 2795de64b1d..a53de155355 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -64,7 +64,10 @@ TEST_P(ParserTest, parseQuery) if (std::string("CREATE USER or ALTER USER query") != parser->getName() && std::string("ATTACH access entity query") != parser->getName()) { - EXPECT_EQ(expected_ast, serializeAST(*ast->clone(), false)); + WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_EQ(expected_ast, formatted_ast); } else { @@ -75,7 +78,10 @@ TEST_P(ParserTest, parseQuery) } else { - EXPECT_TRUE(std::regex_match(serializeAST(*ast->clone(), false), std::regex(expected_ast))); + WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast))); } } } diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index 22484727ea2..c0a975f7a38 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -155,7 +155,7 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties) EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); - EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression, true), "(rand() % 100) * 77"); + EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression), "(rand() % 100) * 77"); EXPECT_EQ(attributes_children[0]->as()->hierarchical, false); EXPECT_EQ(attributes_children[1]->as()->hierarchical, true); @@ -201,7 +201,7 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder) EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); - EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression, true), "(rand() % 100) * 77"); + EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression), "(rand() % 100) * 77"); EXPECT_EQ(attributes_children[0]->as()->hierarchical, false); EXPECT_EQ(attributes_children[1]->as()->hierarchical, true); @@ -288,7 +288,7 @@ TEST(ParserDictionaryDDL, Formatting) ParserCreateDictionaryQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); - auto str = serializeAST(*create, true); + auto str = serializeAST(*create); EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)"); } @@ -303,7 +303,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) EXPECT_TRUE(drop1->is_dictionary); EXPECT_EQ(drop1->getDatabase(), "test"); EXPECT_EQ(drop1->getTable(), "dict1"); - auto str1 = serializeAST(*drop1, true); + auto str1 = serializeAST(*drop1); EXPECT_EQ(input1, str1); String input2 = "DROP DICTIONARY IF EXISTS dict2"; @@ -314,7 +314,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) EXPECT_TRUE(drop2->is_dictionary); EXPECT_EQ(drop2->getDatabase(), ""); EXPECT_EQ(drop2->getTable(), "dict2"); - auto str2 = serializeAST(*drop2, true); + auto str2 = serializeAST(*drop2); EXPECT_EQ(input2, str2); } diff --git a/src/Processors/Transforms/CheckConstraintsTransform.cpp b/src/Processors/Transforms/CheckConstraintsTransform.cpp index 88f02a3926f..3a6595ea4fb 100644 --- a/src/Processors/Transforms/CheckConstraintsTransform.cpp +++ b/src/Processors/Transforms/CheckConstraintsTransform.cpp @@ -73,7 +73,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) "Constraint expression returns nullable column that contains null value", backQuote(constraint_ptr->name), table_id.getNameForLogs(), - serializeAST(*(constraint_ptr->expr), true)); + serializeAST(*(constraint_ptr->expr))); result_column = nested_column; } @@ -116,7 +116,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1, - serializeAST(*(constraint_ptr->expr), true), + serializeAST(*(constraint_ptr->expr)), column_values_msg); } } diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index db37ac7c4c3..249ed8be428 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -35,7 +35,7 @@ String ConstraintsDescription::toString() const for (const auto & constraint : constraints) list.children.push_back(constraint); - return serializeAST(list, true); + return serializeAST(list); } ConstraintsDescription ConstraintsDescription::parse(const String & str) diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index c7aeaf8e4ef..06518a52c61 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -151,7 +151,7 @@ String IndicesDescription::toString() const for (const auto & index : *this) list.children.push_back(index.definition_ast); - return serializeAST(list, true); + return serializeAST(list); } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 73fb279d51c..aecf0ac6d00 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -324,7 +324,7 @@ String ProjectionsDescription::toString() const for (const auto & projection : projections) list.children.push_back(projection.definition_ast); - return serializeAST(list, true); + return serializeAST(list); } ProjectionsDescription ProjectionsDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr query_context) From a24bf14450bdb1dad881330ca168566bf7e1f82f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jul 2023 10:44:44 +0000 Subject: [PATCH 428/473] Use correct ZXID --- src/Coordination/KeeperStateMachine.cpp | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5c84f23fc60..a89b608aa69 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -390,7 +390,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) /// maybe some logs were preprocessed with log idx larger than the snapshot idx /// we have to apply them to the new storage - storage->applyUncommittedState(*snapshot_deserialization_result.storage, s.get_last_log_idx()); + storage->applyUncommittedState(*snapshot_deserialization_result.storage, snapshot_deserialization_result.storage->getZXID()); storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 51a09b676dc..39d9200f913 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -65,7 +65,7 @@ int main(int argc, char *argv[]) CoordinationSettingsPtr settings = std::make_shared(); KeeperContextPtr keeper_context = std::make_shared(true); keeper_context->setLogDisk(std::make_shared("LogDisk", argv[2])); - keeper_context->setSnapshotDisk(std::make_shared("LogDisk", argv[1])); + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", argv[1])); auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); From 27921a5d8f4218a92dafb6fdc145bf3891710e3a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 10:48:39 +0000 Subject: [PATCH 429/473] Docs: Add another reason for integer promotion rules in ClickHouse --- docs/en/sql-reference/functions/arithmetic-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 64fae0e82f0..054c59d5778 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -6,7 +6,7 @@ sidebar_label: Arithmetic # Arithmetic Functions -The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. +The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. Also, this behavior guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer type. The result of addition or multiplication of two integers is unsigned unless one of the integers is signed. From e74acda53ec3a7f8a536eb56e4a939935d10f8e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 20 Jul 2023 12:54:42 +0200 Subject: [PATCH 430/473] PRQL integration (#50686) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Added prql-lib * Add PRQL parser * Extend stateless tests * Add unit tests for `ParserPRQL` --------- Co-authored-by: Ubuntu Co-authored-by: Ubuntu Co-authored-by: Александр Нам <47687537+seshWCS@users.noreply.github.com> --- rust/CMakeLists.txt | 1 + rust/prql/CMakeLists.txt | 3 + rust/prql/Cargo.lock | 569 ++++++++++++++++++ rust/prql/Cargo.toml | 20 + rust/prql/include/prql.h | 18 + rust/prql/src/lib.rs | 56 ++ src/Client/ClientBase.cpp | 4 + src/Common/config.h.in | 1 + src/Core/SettingsEnums.cpp | 4 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/executeQuery.cpp | 7 +- src/Parsers/CMakeLists.txt | 4 + src/Parsers/PRQL/ParserPRQLQuery.cpp | 86 +++ src/Parsers/PRQL/ParserPRQLQuery.h | 27 + src/Parsers/tests/gtest_Parser.cpp | 20 + src/configure_config.cmake | 3 + .../queries/0_stateless/02766_prql.reference | 19 + tests/queries/0_stateless/02766_prql.sh | 58 ++ 18 files changed, 899 insertions(+), 2 deletions(-) create mode 100644 rust/prql/CMakeLists.txt create mode 100644 rust/prql/Cargo.lock create mode 100644 rust/prql/Cargo.toml create mode 100644 rust/prql/include/prql.h create mode 100644 rust/prql/src/lib.rs create mode 100644 src/Parsers/PRQL/ParserPRQLQuery.cpp create mode 100644 src/Parsers/PRQL/ParserPRQLQuery.h create mode 100644 tests/queries/0_stateless/02766_prql.reference create mode 100755 tests/queries/0_stateless/02766_prql.sh diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 6700ead9786..41451fe0a1e 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -88,3 +88,4 @@ endfunction() add_rust_subdirectory (BLAKE3) add_rust_subdirectory (skim) +add_rust_subdirectory (prql) diff --git a/rust/prql/CMakeLists.txt b/rust/prql/CMakeLists.txt new file mode 100644 index 00000000000..65109d19a81 --- /dev/null +++ b/rust/prql/CMakeLists.txt @@ -0,0 +1,3 @@ +clickhouse_import_crate(MANIFEST_PATH Cargo.toml) +target_include_directories(_ch_rust_prql INTERFACE include) +add_library(ch_rust::prql ALIAS _ch_rust_prql) diff --git a/rust/prql/Cargo.lock b/rust/prql/Cargo.lock new file mode 100644 index 00000000000..da94e4ca852 --- /dev/null +++ b/rust/prql/Cargo.lock @@ -0,0 +1,569 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "_ch_rust_prql" +version = "0.1.0" +dependencies = [ + "prql-compiler", + "serde_json", +] + +[[package]] +name = "addr2line" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +dependencies = [ + "getrandom", + "once_cell", + "version_check", +] + +[[package]] +name = "aho-corasick" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" +dependencies = [ + "memchr", +] + +[[package]] +name = "anyhow" +version = "1.0.71" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" +dependencies = [ + "backtrace", +] + +[[package]] +name = "ariadne" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" +dependencies = [ + "unicode-width", + "yansi", +] + +[[package]] +name = "backtrace" +version = "0.3.68" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "cc" +version = "1.0.79" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chumsky" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" +dependencies = [ + "hashbrown 0.12.3", + "stacker", +] + +[[package]] +name = "csv" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +dependencies = [ + "memchr", +] + +[[package]] +name = "either" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" + +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "equivalent" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88bffebc5d80432c9b140ee17875ff173a8ab62faad5b257da912bd2f6c1c0a1" + +[[package]] +name = "getrandom" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.27.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "indexmap" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +dependencies = [ + "equivalent", + "hashbrown 0.14.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62b02a5381cc465bd3041d84623d0fa3b66738b52b8e2fc3bab8ad63ab032f4a" + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "libc" +version = "0.2.147" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" + +[[package]] +name = "log" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" + +[[package]] +name = "memchr" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "object" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" +dependencies = [ + "memchr", +] + +[[package]] +name = "once_cell" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" + +[[package]] +name = "proc-macro2" +version = "1.0.63" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prql-compiler" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" +dependencies = [ + "anyhow", + "ariadne", + "chumsky", + "csv", + "enum-as-inner", + "itertools", + "lazy_static", + "log", + "once_cell", + "regex", + "semver", + "serde", + "serde_json", + "serde_yaml", + "sqlformat", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "psm" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" +dependencies = [ + "cc", +] + +[[package]] +name = "quote" +version = "1.0.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "regex" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89089e897c013b3deb627116ae56a6955a72b8bed395c9526af31c9fe528b484" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa250384981ea14565685dea16a9ccc4d1c541a13f82b9c168572264d1df8c56" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ab07dc67230e4a4718e70fd5c20055a4334b121f1f9db8fe63ef39ce9b8c846" + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustversion" +version = "1.0.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc31bd9b61a32c31f9650d18add92aa83a49ba979c143eefd27fe7177b05bd5f" + +[[package]] +name = "ryu" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe232bdf6be8c8de797b22184ee71118d63780ea42ac85b61d1baa6d3b782ae9" + +[[package]] +name = "semver" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bebd363326d05ec3e2f532ab7660680f3b02130d780c299bca73469d521bc0ed" +dependencies = [ + "serde", +] + +[[package]] +name = "serde" +version = "1.0.166" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d01b7404f9d441d3ad40e6a636a7782c377d2abdbe4fa2440e2edcc2f4f10db8" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.166" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dd83d6dde2b6b2d466e14d9d1acce8816dedee94f735eac6395808b3483c6d6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.23", +] + +[[package]] +name = "serde_json" +version = "1.0.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f1e14e89be7aa4c4b78bdbdc9eb5bf8517829a600ae8eaa39a6e1d960b5185c" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "452e67b9c20c37fa79df53201dc03839651086ed9bbe92b3ca585ca9fdaa7d85" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + +[[package]] +name = "sqlformat" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" +dependencies = [ + "itertools", + "nom", + "unicode_categories", +] + +[[package]] +name = "sqlparser" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +dependencies = [ + "log", + "serde", +] + +[[package]] +name = "stacker" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "winapi", +] + +[[package]] +name = "strum" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.24.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 1.0.109", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "unicode-ident" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22049a19f4a68748a168c0fc439f9516686aa045927ff767eca0a85101fb6e73" + +[[package]] +name = "unicode-width" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" + +[[package]] +name = "unicode_categories" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1865806a559042e51ab5414598446a5871b561d21b6764f2eabb0dd481d880a6" + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/rust/prql/Cargo.toml b/rust/prql/Cargo.toml new file mode 100644 index 00000000000..314d1b52391 --- /dev/null +++ b/rust/prql/Cargo.toml @@ -0,0 +1,20 @@ +[package] +name = "_ch_rust_prql" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +prql-compiler = "0.8.1" +serde_json = "1.0" + +[lib] +crate-type = ["staticlib"] + +[profile.release] +debug = true + +[profile.release-thinlto] +inherits = "release" +lto = true diff --git a/rust/prql/include/prql.h b/rust/prql/include/prql.h new file mode 100644 index 00000000000..29158d7f30d --- /dev/null +++ b/rust/prql/include/prql.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +extern "C" { + +/// Converts a PRQL query to an SQL query. +/// @param query is a pointer to the beginning of the PRQL query. +/// @param size is the size of the PRQL query. +/// @param out is a pointer to a uint8_t pointer which will be set to the beginning of the null terminated SQL query or the error message. +/// @param out_size is the size of the string pointed by `out`. +/// @returns zero in case of success, non-zero in case of failure. +int64_t prql_to_sql(const uint8_t * query, uint64_t size, uint8_t ** out, uint64_t * out_size); + +/// Frees the passed in pointer which's memory was allocated by Rust allocators previously. +void prql_free_pointer(uint8_t * ptr_to_free); + +} // extern "C" diff --git a/rust/prql/src/lib.rs b/rust/prql/src/lib.rs new file mode 100644 index 00000000000..fb71d62d527 --- /dev/null +++ b/rust/prql/src/lib.rs @@ -0,0 +1,56 @@ +use prql_compiler::sql::Dialect; +use prql_compiler::{Options, Target}; +use std::ffi::{c_char, CString}; +use std::slice; + +fn set_output(result: String, out: *mut *mut u8, out_size: *mut u64) { + assert!(!out_size.is_null()); + let out_size_ptr = unsafe { &mut *out_size }; + *out_size_ptr = (result.len() + 1).try_into().unwrap(); + + assert!(!out.is_null()); + let out_ptr = unsafe { &mut *out }; + *out_ptr = CString::new(result).unwrap().into_raw() as *mut u8; +} + +#[no_mangle] +pub unsafe extern "C" fn prql_to_sql( + query: *const u8, + size: u64, + out: *mut *mut u8, + out_size: *mut u64, +) -> i64 { + let query_vec = unsafe { slice::from_raw_parts(query, size.try_into().unwrap()) }.to_vec(); + let maybe_prql_query = String::from_utf8(query_vec); + if maybe_prql_query.is_err() { + set_output( + String::from("The PRQL query must be UTF-8 encoded!"), + out, + out_size, + ); + return 1; + } + let prql_query = maybe_prql_query.unwrap(); + let opts = &Options { + format: true, + target: Target::Sql(Some(Dialect::ClickHouse)), + signature_comment: false, + color: false, + }; + let (is_err, res) = match prql_compiler::compile(&prql_query, &opts) { + Ok(sql_str) => (false, sql_str), + Err(err) => (true, err.to_string()), + }; + + set_output(res, out, out_size); + + match is_err { + true => 1, + false => 0, + } +} + +#[no_mangle] +pub unsafe extern "C" fn prql_free_pointer(ptr_to_free: *mut u8) { + std::mem::drop(CString::from_raw(ptr_to_free as *mut c_char)); +} diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 509dfe2e232..f5390037e6b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include @@ -72,6 +73,7 @@ #include #include #include +#include #include #include "config_version.h" @@ -338,6 +340,8 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu if (dialect == Dialect::kusto) parser = std::make_unique(end, global_context->getSettings().allow_settings_after_format_in_insert); + else if (dialect == Dialect::prql) + parser = std::make_unique(max_length, settings.max_parser_depth); else parser = std::make_unique(end, global_context->getSettings().allow_settings_after_format_in_insert); diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 1cb13d3ae3e..a2c18fc330f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -54,6 +54,7 @@ #cmakedefine01 USE_BORINGSSL #cmakedefine01 USE_BLAKE3 #cmakedefine01 USE_SKIM +#cmakedefine01 USE_PRQL #cmakedefine01 USE_OPENSSL_INTREE #cmakedefine01 USE_ULID #cmakedefine01 FIU_ENABLE diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 1e2cbce9309..86400954e2f 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -138,7 +138,9 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, - {"kusto", Dialect::kusto}}) + {"kusto", Dialect::kusto}, + {"kusto", Dialect::kusto}, + {"prql", Dialect::prql}}) // FIXME: do not add 'kusto_auto' to the list. Maybe remove it from code completely? IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index c2783447441..c61afbd2bbf 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -207,6 +207,7 @@ enum class Dialect clickhouse, kusto, kusto_auto, + prql, }; DECLARE_SETTING_ENUM(Dialect) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b76d20f31d..66bc0bcb757 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -75,6 +75,7 @@ #include #include +#include namespace ProfileEvents { @@ -702,10 +703,14 @@ static std::tuple executeQueryImpl( /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } + else if (settings.dialect == Dialect::prql && !internal) + { + ParserPRQLQuery parser(max_query_size, settings.max_parser_depth); + ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); + } else { ParserQuery parser(end, settings.allow_settings_after_format_in_insert); - /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index d5cf2bd4784..d74137f8a91 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -4,8 +4,12 @@ add_headers_and_sources(clickhouse_parsers .) add_headers_and_sources(clickhouse_parsers ./Access) add_headers_and_sources(clickhouse_parsers ./MySQL) add_headers_and_sources(clickhouse_parsers ./Kusto) +add_headers_and_sources(clickhouse_parsers ./PRQL) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils) +if (TARGET ch_rust::prql) + target_link_libraries(clickhouse_parsers PRIVATE ch_rust::prql) +endif () if (USE_DEBUG_HELPERS) # CMake generator expression will do insane quoting when it encounters special character like quotes, spaces, etc. diff --git a/src/Parsers/PRQL/ParserPRQLQuery.cpp b/src/Parsers/PRQL/ParserPRQLQuery.cpp new file mode 100644 index 00000000000..b3733b727dc --- /dev/null +++ b/src/Parsers/PRQL/ParserPRQLQuery.cpp @@ -0,0 +1,86 @@ +#include +#include + +#include "Parsers/Lexer.h" +#include "config.h" + +#if USE_PRQL +# include +#endif + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int SUPPORT_IS_DISABLED; +} + +bool ParserPRQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserSetQuery set_p; + + if (set_p.parse(pos, node, expected)) + return true; + +#if !USE_PRQL + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, "PRQL is not available. Rust code or PRQL itself may be disabled. Use another dialect!"); +#else + const auto * begin = pos->begin; + + // The same parsers are used in the client and the server, so the parser have to detect the end of a single query in case of multiquery queries + while (!pos->isEnd() && pos->type != TokenType::Semicolon) + ++pos; + + const auto * end = pos->begin; + + uint8_t * sql_query_ptr{nullptr}; + uint64_t sql_query_size{0}; + + const auto res + = prql_to_sql(reinterpret_cast(begin), static_cast(end - begin), &sql_query_ptr, &sql_query_size); + + SCOPE_EXIT({ prql_free_pointer(sql_query_ptr); }); + + const auto * sql_query_char_ptr = reinterpret_cast(sql_query_ptr); + const auto * const original_sql_query_ptr = sql_query_char_ptr; + + if (res != 0) + { + throw Exception(ErrorCodes::SYNTAX_ERROR, "PRQL syntax error: '{}'", sql_query_char_ptr); + } + chassert(sql_query_size > 0); + + ParserQuery query_p(end, false); + String error_message; + node = tryParseQuery( + query_p, + sql_query_char_ptr, + sql_query_char_ptr + sql_query_size - 1, + error_message, + false, + "", + false, + max_query_size, + max_parser_depth); + + if (!node) + throw Exception( + ErrorCodes::SYNTAX_ERROR, + "Error while parsing the SQL query generated from PRQL query :'{}'.\nPRQL Query:'{}'\nSQL query: '{}'", + error_message, + std::string_view{begin, end}, + std::string_view(original_sql_query_ptr, original_sql_query_ptr + sql_query_size)); + + + return true; +#endif +} +} diff --git a/src/Parsers/PRQL/ParserPRQLQuery.h b/src/Parsers/PRQL/ParserPRQLQuery.h new file mode 100644 index 00000000000..4fc450df6b6 --- /dev/null +++ b/src/Parsers/PRQL/ParserPRQLQuery.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +namespace DB +{ +// Even when PRQL is disabled, it is not possible to exclude this parser because changing the dialect via `SET dialect = '...'` queries should succeed. +// Another solution would be disabling setting the dialect to PRQL, but it requires a lot of finicky conditional compiling around the Dialect setting enum. +// Therefore the decision, for now, is to use this parser even when PRQL is disabled to enable users to switch to another dialect. +class ParserPRQLQuery final : public IParserBase +{ +private: + // These fields are not used when PRQL is disabled at build time. + [[maybe_unused]] size_t max_query_size; + [[maybe_unused]] size_t max_parser_depth; + +public: + ParserPRQLQuery(size_t max_query_size_, size_t max_parser_depth_) : max_query_size{max_query_size_}, max_parser_depth{max_parser_depth_} + { + } + + const char * getName() const override { return "PRQL Statement"; } + +protected: + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 2795de64b1d..ef4ef05e35e 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -476,3 +477,22 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, "SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE 'pet%')" } }))); + +static constexpr size_t kDummyMaxQuerySize = 256 * 1024; +static constexpr size_t kDummyMaxParserDepth = 256; + +INSTANTIATE_TEST_SUITE_P( + ParserPRQL, + ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared(kDummyMaxQuerySize, kDummyMaxParserDepth)), + ::testing::ValuesIn(std::initializer_list{ + { + "from albums\ngroup [author_id] (\n aggregate [first_pushlied = min published]\n)\njoin a=author side:left [==author_id]\njoin p=purchases side:right [==author_id]\ngroup [a.id, p.purchase_id] (\n aggregate [avg_sell = min first_pushlied]\n)", + "WITH table_1 AS\n (\n SELECT\n MIN(published) AS _expr_0,\n author_id\n FROM albums\n GROUP BY author_id\n )\nSELECT\n a.id,\n p.purchase_id,\n MIN(table_0._expr_0) AS avg_sell\nFROM table_1 AS table_0\nLEFT JOIN author AS a ON table_0.author_id = a.author_id\nRIGHT JOIN purchases AS p ON table_0.author_id = p.author_id\nGROUP BY\n a.id,\n p.purchase_id", + }, + { + "from matches\nfilter start_date > @2023-05-30 # Some comment here\nderive [\n some_derived_value_1 = a + (b ?? 0), # And there\n some_derived_value_2 = c + some_derived_value\n]\nfilter some_derived_value_2 > 0\ngroup [country, city] (\n aggregate [\n average some_derived_value_2,\n aggr = max some_derived_value_2,\n ]\n)\nderive place = f\"{city} in {country}\"\nderive country_code = s\"LEFT(country, 2)\"\nsort [aggr, -country]\ntake 1..20", + "WITH\n table_3 AS\n (\n SELECT\n country,\n city,\n c + some_derived_value AS _expr_1\n FROM matches\n WHERE start_date > toDate('2023-05-30')\n ),\n table_1 AS\n (\n SELECT\n country,\n city,\n AVG(_expr_1) AS _expr_0,\n MAX(_expr_1) AS aggr\n FROM table_3 AS table_2\n WHERE _expr_1 > 0\n GROUP BY\n country,\n city\n )\nSELECT\n country,\n city,\n _expr_0,\n aggr,\n CONCAT(city, ' in ', country) AS place,\n LEFT(country, 2) AS country_code\nFROM table_1 AS table_0\nORDER BY\n aggr ASC,\n country DESC\nLIMIT 20", + }, + }))); diff --git a/src/configure_config.cmake b/src/configure_config.cmake index c11a19b36ea..ae6305705c2 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -25,6 +25,9 @@ endif() if (TARGET ch_rust::skim) set(USE_SKIM 1) endif() +if (TARGET ch_rust::prql) + set(USE_PRQL 1) +endif() if (TARGET OpenSSL::SSL) set(USE_SSL 1) endif() diff --git a/tests/queries/0_stateless/02766_prql.reference b/tests/queries/0_stateless/02766_prql.reference new file mode 100644 index 00000000000..90e0b26cee6 --- /dev/null +++ b/tests/queries/0_stateless/02766_prql.reference @@ -0,0 +1,19 @@ +101 Hello, ClickHouse! 2 He +101 Granules are the smallest chunks of data read 2 Gr +102 Insert a lot of rows per batch 2 In +102 Sort your data based on your commonly-used queries 2 So +103 This is an awesome message 2 Th +103 42 +102 4.132209897041321 +--- +101 Hello, ClickHouse! 2019-01-01 00:00:00.000 -1 +101 Granules are the smallest chunks of data read 2019-05-01 00:00:00.000 3.14159 +102 Insert a lot of rows per batch 2019-02-01 00:00:00.000 1.41421 +102 Sort your data based on your commonly-used queries 2019-03-01 00:00:00.000 2.718 +103 This is an awesome message 2019-04-01 00:00:00.000 42 +--- +101 Hello, ClickHouse! 2019-01-01 00:00:00.000 -1 +101 Granules are the smallest chunks of data read 2019-05-01 00:00:00.000 3.14159 +102 Insert a lot of rows per batch 2019-02-01 00:00:00.000 1.41421 +102 Sort your data based on your commonly-used queries 2019-03-01 00:00:00.000 2.718 +103 This is an awesome message 2019-04-01 00:00:00.000 42 diff --git a/tests/queries/0_stateless/02766_prql.sh b/tests/queries/0_stateless/02766_prql.sh new file mode 100755 index 00000000000..f8bbd72af4e --- /dev/null +++ b/tests/queries/0_stateless/02766_prql.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " +CREATE TEMPORARY TABLE IF NOT EXISTS aboba +( + user_id UInt32, + message String, + creation_date DateTime64, + metric Float32 +) +ENGINE = MergeTree +ORDER BY user_id; + +INSERT INTO aboba (user_id, message, creation_date, metric) VALUES (101, 'Hello, ClickHouse!', toDateTime('2019-01-01 00:00:00', 3, 'Europe/Amsterdam'), -1.0), (102, 'Insert a lot of rows per batch', toDateTime('2019-02-01 00:00:00', 3, 'Europe/Amsterdam'), 1.41421 ), (102, 'Sort your data based on your commonly-used queries', toDateTime('2019-03-01 00:00:00', 3, 'Europe/Amsterdam'), 2.718), (101, 'Granules are the smallest chunks of data read', toDateTime('2019-05-01 00:00:00', 3, 'Europe/Amsterdam'), 3.14159), (103, 'This is an awesome message', toDateTime('2019-04-01 00:00:00', 3, 'Europe/Amsterdam'), 42); + +SET dialect = 'prql'; + +from aboba +derive [ + a = 2, + b = s\"LEFT(message, 2)\" +] +select [ user_id, message, a, b ]; + +from aboba +filter user_id > 101 +group user_id ( + aggregate [ + metrics = sum metric + ] +); + +SET dialect = 'clickhouse'; + +SELECT '---'; +SELECT + user_id, + message, + toTimeZone(creation_date, 'Europe/Amsterdam') as creation_date, + metric +FROM aboba; +SELECT '---'; + +SET dialect = 'prql'; + +from aboba +select [ user_id, message, metric ] +derive creation_date = s\"toTimeZone(creation_date, 'Europe/Amsterdam')\" +select [ user_id, message, creation_date, metric]; + +from s\"SELECT * FROM system.users\" | select non_existent_column; # {serverError UNKNOWN_IDENTIFIER} +from non_existent_table; # {serverError UNKNOWN_TABLE} +" \ No newline at end of file From 84f6a7336c2d7ac547ad7030c389d4961f4ab8e4 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 19:03:42 +0800 Subject: [PATCH 431/473] Prevent going beyond the index of const_columns_to_remove. --- src/Processors/Transforms/FinishSortingTransform.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 744d035d0ee..baf898481ab 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -38,11 +38,12 @@ FinishSortingTransform::FinishSortingTransform( /// Remove constants from description_sorted_. SortDescription description_sorted_without_constants; description_sorted_without_constants.reserve(description_sorted_.size()); + size_t num_columns = const_columns_to_remove.size(); for (const auto & column_description : description_sorted_) { auto pos = header.getPositionByName(column_description.column_name); - if (!const_columns_to_remove[pos]){ + if (pos < num_columns && !const_columns_to_remove[pos]){ description_sorted_without_constants.push_back(column_description); } } From f2d184cf1b002d18be152880ee2d82e57fed3b26 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 19:11:08 +0800 Subject: [PATCH 432/473] Consistent style for if statements. --- src/Processors/Transforms/FinishSortingTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index baf898481ab..63a9c3924a2 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -43,9 +43,8 @@ FinishSortingTransform::FinishSortingTransform( { auto pos = header.getPositionByName(column_description.column_name); - if (pos < num_columns && !const_columns_to_remove[pos]){ + if (pos < num_columns && !const_columns_to_remove[pos]) description_sorted_without_constants.push_back(column_description); - } } /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. From db1b53d1bb8ed6aa71f47010c81a7f3ebb0ae65d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:18:48 +0300 Subject: [PATCH 433/473] Update 01606_git_import.sh --- tests/queries/0_stateless/01606_git_import.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01606_git_import.sh b/tests/queries/0_stateless/01606_git_import.sh index c9aa2c7d82e..48558d79f93 100755 --- a/tests/queries/0_stateless/01606_git_import.sh +++ b/tests/queries/0_stateless/01606_git_import.sh @@ -13,7 +13,7 @@ cd $CLICKHOUSE_TMP || exit # Protection for network errors for _ in {1..10}; do rm -rf ./clickhouse-odbc - git clone --quiet https://github.com/ClickHouse/clickhouse-odbc.git && pushd clickhouse-odbc > /dev/null && git checkout --quiet 5d84ec591c53cbb272593f024230a052690fdf69 && break + git clone --quiet https://github.com/ClickHouse/clickhouse-odbc.git && pushd clickhouse-odbc 2> /dev/null > /dev/null && git checkout --quiet 5d84ec591c53cbb272593f024230a052690fdf69 && break sleep 1 done From 2b29e3dc83d9ed6747acb4a249c9e1aca9616f21 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:22:22 +0300 Subject: [PATCH 434/473] Update MergeTreeBackgroundExecutor.cpp (#52261) --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 6eab4337162..e497a799274 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -269,7 +269,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) try { ALLOW_ALLOCATIONS_IN_SCOPE; - item->task->getQueryId(); + query_id = item->task->getQueryId(); need_execute_again = item->task->executeStep(); } catch (...) From f53ff5d4f2228b7016af5742ea1ae8f70ef772df Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:51:01 +0300 Subject: [PATCH 435/473] more fair queue for drop table sync (#52276) --- src/Interpreters/DatabaseCatalog.cpp | 17 ++++++++++++++++- src/Interpreters/DatabaseCatalog.h | 1 + 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 23a67f4bc2f..0e2e30eefee 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -697,6 +697,7 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(&Poco::Logger::get("DatabaseCatalog")) + , first_async_drop_in_queue(tables_marked_dropped.end()) { } @@ -959,9 +960,17 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr std::lock_guard lock(tables_marked_dropped_mutex); if (ignore_delay) - tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, drop_time}); + { + /// Insert it before first_async_drop_in_queue, so sync drop queries will have priority over async ones, + /// but the queue will remain fair for multiple sync drop queries. + tables_marked_dropped.emplace(first_async_drop_in_queue, TableMarkedAsDropped{table_id, table, dropped_metadata_path, drop_time}); + } else + { tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time + drop_delay_sec}); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; + } tables_marked_dropped_ids.insert(table_id.uuid); CurrentMetrics::add(CurrentMetrics::TablesToDropQueueSize, 1); @@ -1012,6 +1021,8 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) /// This maybe throw exception. renameNoReplace(latest_metadata_dropped_path, table_metadata_path); + if (first_async_drop_in_queue == it_dropped_table) + ++first_async_drop_in_queue; tables_marked_dropped.erase(it_dropped_table); [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table.table_id.uuid); assert(removed); @@ -1074,6 +1085,8 @@ void DatabaseCatalog::dropTableDataTask() table = std::move(*it); LOG_INFO(log, "Have {} tables in drop queue ({} of them are in use), will try drop {}", tables_marked_dropped.size(), tables_in_use_count, table.table_id.getNameForLogs()); + if (first_async_drop_in_queue == it) + ++first_async_drop_in_queue; tables_marked_dropped.erase(it); /// Schedule the task as soon as possible, while there are suitable tables to drop. schedule_after_ms = 0; @@ -1110,6 +1123,8 @@ void DatabaseCatalog::dropTableDataTask() table.drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; std::lock_guard lock(tables_marked_dropped_mutex); tables_marked_dropped.emplace_back(std::move(table)); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; /// If list of dropped tables was empty, schedule a task to retry deletion. if (tables_marked_dropped.size() == 1) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index d502505027f..805d7786569 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -323,6 +323,7 @@ private: mutable std::mutex ddl_guards_mutex; TablesMarkedAsDropped tables_marked_dropped TSA_GUARDED_BY(tables_marked_dropped_mutex); + TablesMarkedAsDropped::iterator first_async_drop_in_queue TSA_GUARDED_BY(tables_marked_dropped_mutex); std::unordered_set tables_marked_dropped_ids TSA_GUARDED_BY(tables_marked_dropped_mutex); mutable std::mutex tables_marked_dropped_mutex; From 3c9e46b557a882085fdcdce5d74ad12329457db3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 16:19:12 +0300 Subject: [PATCH 436/473] Update ci-slack-bot.py --- utils/ci-slack-bot/ci-slack-bot.py | 43 ++++++++++++++++++++++++++---- 1 file changed, 38 insertions(+), 5 deletions(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 6e694b4fdbd..0fb12e89ce9 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -26,10 +26,11 @@ else: DRY_RUN_MARK = "" -MAX_FAILURES_DEFAULT = 40 +MAX_FAILURES_DEFAULT = 30 SLACK_URL_DEFAULT = DRY_RUN_MARK -FLAKY_ALERT_PROBABILITY = 0.20 +FLAKY_ALERT_PROBABILITY = 0.50 +REPORT_NO_FAILURES_PROBABILITY = 0.99 MAX_TESTS_TO_REPORT = 4 @@ -89,6 +90,22 @@ WHERE 1 AND check_name ILIKE check_name_pattern """ +# Returns percentage of failed checks (once per day, at noon) +FAILED_CHECKS_PERCENTAGE_QUERY = """ +SELECT if(toHour(now('Europe/Amsterdam')) = 12, v, 0) +FROM +( + SELECT + countDistinctIf((commit_sha, check_name), (test_status LIKE 'F%') AND (check_status != 'success')) + / countDistinct((commit_sha, check_name)) AS v + FROM checks + WHERE 1 + AND (pull_request_number = 0) + AND (test_status != 'SKIPPED') + AND (check_start_time > (now() - toIntervalDay(1))) +) +""" + # It shows all recent failures of the specified test (helps to find when it started) ALL_RECENT_FAILURES_QUERY = """ WITH @@ -202,9 +219,9 @@ def get_too_many_failures_message_impl(failures_count): curr_failures = int(failures_count[0][0]) prev_failures = int(failures_count[0][1]) if curr_failures == 0 and prev_failures != 0: - return ( - "Looks like CI is completely broken: there are *no failures* at all... 0_o" - ) + if random.random() < REPORT_NO_FAILURES_PROBABILITY: + return None + return "Wow, there are *no failures* at all... 0_o" if curr_failures < MAX_FAILURES: return None if prev_failures < MAX_FAILURES: @@ -227,6 +244,19 @@ def get_too_many_failures_message(failures_count): return msg +def get_failed_checks_percentage_message(percentage): + p = percentage[0][0] * 100 + + # Always report more than 1% of failed checks + # For <= 1%: higher percentage of failures == higher probability + if p <= random.random(): + return None + + msg = ":alert: " if p > 1 else "Only " if p < 0.5 else "" + msg += "*{0:.2f}%* of all checks in master have failed yesterday".format(p) + return msg + + def split_slack_message(long_message): lines = long_message.split("\n") messages = [] @@ -280,6 +310,9 @@ def query_and_alert_if_needed(query, get_message_func): def check_and_alert(): query_and_alert_if_needed(NEW_BROKEN_TESTS_QUERY, get_new_broken_tests_message) query_and_alert_if_needed(COUNT_FAILURES_QUERY, get_too_many_failures_message) + query_and_alert_if_needed( + FAILED_CHECKS_PERCENTAGE_QUERY, get_failed_checks_percentage_message + ) def lambda_handler(event, context): From f997adfe27e1bd3bb772857fb11fae962c373b9e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jul 2023 14:02:55 +0000 Subject: [PATCH 437/473] Retry if sessions not closed because missing leader --- src/Coordination/KeeperDispatcher.cpp | 37 +++++++++------- tests/integration/test_keeper_session/test.py | 42 ++++++++++++++----- 2 files changed, 54 insertions(+), 25 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9d9df5c7f30..dfb621eb0ad 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -473,23 +473,30 @@ void KeeperDispatcher::shutdown() session_to_response_callback.clear(); } - // if there is no leader, there is no reason to do CLOSE because it's a write request - if (server && hasLeader() && !close_requests.empty()) + if (server && !close_requests.empty()) { - LOG_INFO(log, "Trying to close {} session(s)", close_requests.size()); - const auto raft_result = server->putRequestBatch(close_requests); - auto sessions_closing_done_promise = std::make_shared>(); - auto sessions_closing_done = sessions_closing_done_promise->get_future(); - raft_result->when_ready([my_sessions_closing_done_promise = std::move(sessions_closing_done_promise)]( - nuraft::cmd_result> & /*result*/, - nuraft::ptr & /*exception*/) { my_sessions_closing_done_promise->set_value(); }); + // if there is no leader, there is no reason to do CLOSE because it's a write request + if (hasLeader()) + { + LOG_INFO(log, "Trying to close {} session(s)", close_requests.size()); + const auto raft_result = server->putRequestBatch(close_requests); + auto sessions_closing_done_promise = std::make_shared>(); + auto sessions_closing_done = sessions_closing_done_promise->get_future(); + raft_result->when_ready([my_sessions_closing_done_promise = std::move(sessions_closing_done_promise)]( + nuraft::cmd_result> & /*result*/, + nuraft::ptr & /*exception*/) { my_sessions_closing_done_promise->set_value(); }); - auto session_shutdown_timeout = configuration_and_settings->coordination_settings->session_shutdown_timeout.totalMilliseconds(); - if (sessions_closing_done.wait_for(std::chrono::milliseconds(session_shutdown_timeout)) != std::future_status::ready) - LOG_WARNING( - log, - "Failed to close sessions in {}ms. If they are not closed, they will be closed after session timeout.", - session_shutdown_timeout); + auto session_shutdown_timeout = configuration_and_settings->coordination_settings->session_shutdown_timeout.totalMilliseconds(); + if (sessions_closing_done.wait_for(std::chrono::milliseconds(session_shutdown_timeout)) != std::future_status::ready) + LOG_WARNING( + log, + "Failed to close sessions in {}ms. If they are not closed, they will be closed after session timeout.", + session_shutdown_timeout); + } + else + { + LOG_INFO(log, "Sessions cannot be closed during shutdown because there is no active leader"); + } } if (server) diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py index e57057a8258..68147865cd2 100644 --- a/tests/integration/test_keeper_session/test.py +++ b/tests/integration/test_keeper_session/test.py @@ -6,6 +6,7 @@ import socket import struct from kazoo.client import KazooClient +from kazoo.exceptions import NoNodeError # from kazoo.protocol.serialization import Connect, read_buffer, write_buffer @@ -162,17 +163,38 @@ def test_session_timeout(started_cluster): def test_session_close_shutdown(started_cluster): wait_nodes() - node1_zk = get_fake_zk(node1.name) - node2_zk = get_fake_zk(node2.name) + node1_zk = None + node2_zk = None + for i in range(20): + node1_zk = get_fake_zk(node1.name) + node2_zk = get_fake_zk(node2.name) - eph_node = "/test_node" - node2_zk.create(eph_node, ephemeral=True) - node1_zk.sync(eph_node) - assert node1_zk.exists(eph_node) != None + eph_node = "/test_node" + node2_zk.create(eph_node, ephemeral=True) + node1_zk.sync(eph_node) - # shutdown while session is active - node2.stop_clickhouse() + node1_zk.exists(eph_node) != None - assert node1_zk.exists(eph_node) == None + # restart while session is active so it's closed during shutdown + node2.restart_clickhouse() - node2.start_clickhouse() + if node1_zk.exists(eph_node) == None: + break + + assert node2.contains_in_log("Sessions cannot be closed during shutdown because there is no active leader") + + try: + node1_zk.delete(eph_node) + except NoNodeError: + pass + + assert node1_zk.exists(eph_node) == None + + destroy_zk_client(node1_zk) + node1_zk = None + destroy_zk_client(node2_zk) + node2_zk = None + + time.sleep(1) + else: + assert False, "Session wasn't properly cleaned up on shutdown" \ No newline at end of file From 5decb1f5c555d2465724f9bc3c555c157f9deb81 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 20 Jul 2023 14:11:11 +0000 Subject: [PATCH 438/473] Automatic style fix --- tests/integration/test_keeper_session/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py index 68147865cd2..cd012ad6e9e 100644 --- a/tests/integration/test_keeper_session/test.py +++ b/tests/integration/test_keeper_session/test.py @@ -181,7 +181,9 @@ def test_session_close_shutdown(started_cluster): if node1_zk.exists(eph_node) == None: break - assert node2.contains_in_log("Sessions cannot be closed during shutdown because there is no active leader") + assert node2.contains_in_log( + "Sessions cannot be closed during shutdown because there is no active leader" + ) try: node1_zk.delete(eph_node) @@ -197,4 +199,4 @@ def test_session_close_shutdown(started_cluster): time.sleep(1) else: - assert False, "Session wasn't properly cleaned up on shutdown" \ No newline at end of file + assert False, "Session wasn't properly cleaned up on shutdown" From 046bf55dc084d4df91ecfddb8e22aa6f9300fa43 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 14:17:33 +0000 Subject: [PATCH 439/473] Incorporate feedback --- .../functions/arithmetic-functions.md | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 054c59d5778..69f1816b7df 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -6,9 +6,20 @@ sidebar_label: Arithmetic # Arithmetic Functions -The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. Also, this behavior guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer type. +Arithmetic functions work for any two operands of type `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64`. -The result of addition or multiplication of two integers is unsigned unless one of the integers is signed. +Before performing the operation, both operands are casted to the result type. The result type is determined as follows (unless specified +differently in the function documentation below): +- If both operands are up to 32 bits wide, the size of the result type will be the size of the next bigger type following the bigger of the + two operands (integer size promotion). For example, `UInt8 + UInt16 = UInt32` or `Float32 * Float32 = Float64`. +- If one of the operands has 64 or more bits, the size of the result type will be the same size as the bigger of the two operands. For + example, `UInt32 + UInt128 = UInt128` or `Float32 * Float64 = Float64`. +- If one of the operands is signed, the result type will also be signed, otherwise it will be signed. For example, `UInt32 * Int32 = Int64`. + +These rules make sure that the result type will be the smallest type which can represent all possible results. While this introduces a risk +of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of +64 bit. This behavior also guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer +type. Example: @@ -22,8 +33,6 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 └───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘ ``` -Arithmetic functions work for any pair of `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64` values. - Overflows are produced the same way as in C++. ## plus @@ -68,7 +77,7 @@ Alias: `a \* b` (operator) ## divide -Calculates the quotient of two values `a` and `b`. The result is always a floating-point value. If you need integer division, you can use the `intDiv` function. +Calculates the quotient of two values `a` and `b`. The result type is always [Float64](../../sql-reference/data-types/float.md). Integer division is provided by the `intDiv` function. Division by 0 returns `inf`, `-inf`, or `nan`. @@ -84,7 +93,7 @@ Alias: `a / b` (operator) Performs an integer division of two values `a` by `b`, i.e. computes the quotient rounded down to the next smallest integer. -The result has the same type as the dividend (the first parameter). +The result has the same width as the dividend (the first parameter). An exception is thrown when dividing by zero, when the quotient does not fit in the range of the dividend, or when dividing a minimal negative number by minus one. @@ -135,7 +144,7 @@ intDivOrZero(a, b) Calculates the remainder of the division of two values `a` by `b`. -The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result is a floating-point number. +The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result type is [Float64](../../sql-reference/data-types/float.md). The remainder is computed like in C++. Truncated division is used for negative numbers. From e467264588a6435199879fd89d1dc995c9e37c63 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 17:56:30 +0300 Subject: [PATCH 440/473] Update src/IO/HTTPCommon.cpp --- src/IO/HTTPCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index a5816911c09..ddd7ccbe483 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -81,7 +81,7 @@ namespace Session::close(); LOG_TRACE( log, - "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", + "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", Session::getResolvedHost(), Session::getHost(), Session::getPort()); From c0aa3e456705e3ef75ed09683f4e9ed6d9151917 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 17:59:43 +0300 Subject: [PATCH 441/473] Update ci-slack-bot.py --- utils/ci-slack-bot/ci-slack-bot.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 0fb12e89ce9..ea883e3cda3 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -245,7 +245,7 @@ def get_too_many_failures_message(failures_count): def get_failed_checks_percentage_message(percentage): - p = percentage[0][0] * 100 + p = float(percentage[0][0]) * 100 # Always report more than 1% of failed checks # For <= 1%: higher percentage of failures == higher probability From ea252e2f612afd9e83c1aa000af945eebbe18a16 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 15:05:07 +0000 Subject: [PATCH 442/473] Disable analyzer setting in backward_compatibility integration tests. --- tests/integration/helpers/cluster.py | 6 +++++- tests/integration/test_backward_compatibility/test.py | 2 ++ .../test_aggregate_fixed_key.py | 5 +++-- .../test_aggregate_function_state.py | 6 ++++-- .../test_backward_compatibility/test_convert_ordinary.py | 1 + .../test_backward_compatibility/test_cte_distributed.py | 7 ++++--- .../test_data_skipping_indices.py | 1 + .../test_backward_compatibility/test_functions.py | 3 ++- .../test_in_memory_parts_still_read.py | 1 + .../test_insert_profile_events.py | 3 ++- .../test_ip_types_binary_compatibility.py | 1 + .../test_memory_bound_aggregation.py | 4 +++- .../test_normalized_count_comparison.py | 3 ++- .../test_select_aggregate_alias_column.py | 3 ++- .../test_short_strings_aggregation.py | 4 +++- .../test_vertical_merges_from_compact_parts.py | 2 ++ 16 files changed, 38 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c52442ecb9c..c85fbb8ad9e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1533,6 +1533,7 @@ class ClickHouseCluster: with_jdbc_bridge=False, with_hive=False, with_coredns=False, + allow_analyzer=True, hostname=None, env_variables=None, image="clickhouse/integration-test", @@ -1630,6 +1631,7 @@ class ClickHouseCluster: with_hive=with_hive, with_coredns=with_coredns, with_cassandra=with_cassandra, + allow_analyzer=allow_analyzer, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -3169,6 +3171,7 @@ class ClickHouseInstance: with_hive, with_coredns, with_cassandra, + allow_analyzer, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, @@ -3256,6 +3259,7 @@ class ClickHouseInstance: self.with_hive = with_hive self.with_coredns = with_coredns self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) + self.allow_analyzer = allow_analyzer self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -4227,7 +4231,7 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None: + if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None and self.allow_analyzer: write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index ea1d3ab9c07..c3d3b8aad34 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,11 +10,13 @@ node1 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, + allow_analyzer=False ) node2 = cluster.add_instance( "node2", main_configs=["configs/wide_parts_only.xml", "configs/no_compress_marks.xml"], with_zookeeper=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 01c9736c354..cf258987cbf 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -9,9 +9,10 @@ node1 = cluster.add_instance( image="yandex/clickhouse-server", tag="21.3", with_installed_binary=True, + allow_analyzer=False, ) -node2 = cluster.add_instance("node2", with_zookeeper=True) -node3 = cluster.add_instance("node3", with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True, allow_analyzer=False) +node3 = cluster.add_instance("node3", with_zookeeper=True, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py index 1f6d405603a..3a936239cc8 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,9 +19,10 @@ node2 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) -node4 = cluster.add_instance("node4", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) +node4 = cluster.add_instance("node4", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 8b1afd358eb..36facdd59b1 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -9,6 +9,7 @@ node = cluster.add_instance( stay_alive=True, with_zookeeper=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index 7ea0d2d9f21..c68468aad75 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.3.14", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) @@ -31,7 +32,7 @@ WITH quantile(0.05)(cnt) as p05, quantile(0.95)(cnt) as p95, p95 - p05 as inter_percentile_range -SELECT +SELECT sum(cnt) as total_requests, count() as data_points, inter_percentile_range @@ -49,7 +50,7 @@ WITH quantile(0.05)(cnt) as p05, quantile(0.95)(cnt) as p95, p95 - p05 as inter_percentile_range -SELECT +SELECT sum(cnt) as total_requests, count() as data_points, inter_percentile_range diff --git a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py index c65dc6d3841..46ab27d2ab0 100644 --- a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py +++ b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py @@ -12,6 +12,7 @@ node = cluster.add_instance( tag="21.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index afb19901e74..fa24b146fec 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -upstream = cluster.add_instance("upstream") +upstream = cluster.add_instance("upstream", allow_analyzer=False) backward = cluster.add_instance( "backward", image="clickhouse/clickhouse-server", @@ -19,6 +19,7 @@ backward = cluster.add_instance( # Affected at least: singleValueOrNull, last_value, min, max, any, anyLast, anyHeavy, first_value, argMin, argMax tag="22.6", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py b/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py index d55f155918e..cd67f1f6344 100644 --- a/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py +++ b/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py @@ -12,6 +12,7 @@ node = cluster.add_instance( tag="23.4", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 0fd453e57d4..8564c6b5952 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -7,12 +7,13 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -upstream_node = cluster.add_instance("upstream_node") +upstream_node = cluster.add_instance("upstream_node", allow_analyzer=False) old_node = cluster.add_instance( "old_node", image="clickhouse/clickhouse-server", tag="22.5.1.2079", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py index bb40dff27ac..04016755a24 100644 --- a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py +++ b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py @@ -10,6 +10,7 @@ node_22_6 = cluster.add_instance( tag="22.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index d76c4eba409..96b41c81384 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="21.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,8 +19,9 @@ node2 = cluster.add_instance( tag="21.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py index fcdedd29dad..3cd708d5029 100644 --- a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py +++ b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.2.7", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index 8bdae54a889..7e10b6ab430 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.2.7", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 17a7282b7b5..e4fda618031 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,8 +19,9 @@ node2 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 3d006caad0d..82ffcc20b60 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -11,12 +11,14 @@ node_old = cluster.add_instance( stay_alive=True, with_installed_binary=True, with_zookeeper=True, + allow_analyzer=False, ) node_new = cluster.add_instance( "node2", main_configs=["configs/no_compress_marks.xml"], with_zookeeper=True, stay_alive=True, + allow_analyzer=False, ) From a26de1b370e8c09c548528ffbe3337cbf2340012 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 20 Jul 2023 15:12:55 +0000 Subject: [PATCH 443/473] Automatic style fix --- tests/integration/helpers/cluster.py | 5 ++++- tests/integration/test_backward_compatibility/test.py | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c85fbb8ad9e..0ac2f330b1e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4231,7 +4231,10 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None and self.allow_analyzer: + if ( + os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None + and self.allow_analyzer + ): write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index c3d3b8aad34..6f21b184a95 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, - allow_analyzer=False + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", From e6624a07e4fe938b55dd6bc5d8cbabd0ed93d2d7 Mon Sep 17 00:00:00 2001 From: AlexBykovski Date: Thu, 20 Jul 2023 18:54:48 +0300 Subject: [PATCH 444/473] Update build-osx.md syntax error in command for compiler for OSx compilation --- docs/ru/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/build-osx.md b/docs/ru/development/build-osx.md index 9a1f9c9347d..6b4e612b13f 100644 --- a/docs/ru/development/build-osx.md +++ b/docs/ru/development/build-osx.md @@ -68,7 +68,7 @@ $ /bin/bash -c "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/ $ rm -rf build $ mkdir build $ cd build - $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER==$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. + $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. $ cmake --build . --config RelWithDebInfo $ cd .. From 97e54d6ebaa174f8d2ae291ddec20fd879b29bfa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:13:25 +0000 Subject: [PATCH 445/473] Fix test_backup_restore_on_cluster flakiness caused by missing replica syncs --- tests/integration/test_backup_restore_on_cluster/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 6af3a7dbab8..39496b8a5c8 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -580,6 +580,7 @@ def test_required_privileges(): node1.query( f"RESTORE TABLE tbl AS tbl2 ON CLUSTER 'cluster' FROM {backup_name}", user="u1" ) + node2.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl2") assert node2.query("SELECT * FROM tbl2") == "100\n" @@ -593,6 +594,7 @@ def test_required_privileges(): node1.query("GRANT INSERT, CREATE TABLE ON tbl TO u1") node1.query(f"RESTORE ALL ON CLUSTER 'cluster' FROM {backup_name}", user="u1") + node2.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") assert node2.query("SELECT * FROM tbl") == "100\n" From 13f8d72f54433a790f3efcb054db389e4fdd53f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jul 2023 17:46:22 +0200 Subject: [PATCH 446/473] Wait for zero copy replication lock even if some disks don't support it --- .../MergeTree/MergeFromLogEntryTask.cpp | 8 +++++-- .../MergeTree/MutateFromLogEntryTask.cpp | 6 ++++- .../ReplicatedMergeMutateTaskBase.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 22 +++++++++++++++---- 4 files changed, 30 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 9f54c554c85..883cfee89c8 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -230,7 +230,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() /// the fast replica is not overloaded because amount of executing merges doesn't affect the ability to acquire locks for new merges. /// /// So here we trying to solve it with the simplest solution -- sleep random time up to 500ms for 1GB part and up to 7 seconds for 300GB part. - /// It can sound too much, but we are trying to aquite these locks in background tasks which can be scheduled each 5 seconds or so. + /// It can sound too much, but we are trying to acquire these locks in background tasks which can be scheduled each 5 seconds or so. double start_to_sleep_seconds = std::logf(storage_settings_ptr->zero_copy_merge_mutation_min_parts_size_sleep_before_lock.value); uint64_t right_border_to_sleep_ms = static_cast((std::log(estimated_space_for_merge) - start_to_sleep_seconds + 0.5) * 1000); uint64_t time_to_sleep_milliseconds = std::min(10000UL, std::uniform_int_distribution(1, 1 + right_border_to_sleep_ms)(rng)); @@ -245,7 +245,11 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() if (!zero_copy_lock || !zero_copy_lock->isLocked()) { - LOG_DEBUG(log, "Merge of part {} started by some other replica, will wait it and fetch merged part", entry.new_part_name); + LOG_DEBUG( + log, + "Merge of part {} started by some other replica, will wait for it and fetch merged part. Number of tries {}", + entry.new_part_name, + entry.num_tries); storage.watchZeroCopyLock(entry.new_part_name, disk); /// Don't check for missing part -- it's missing because other replica still not /// finished merge. diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 6cb9d50436e..164b541d2b8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -154,8 +154,12 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() if (!zero_copy_lock || !zero_copy_lock->isLocked()) { + LOG_DEBUG( + log, + "Mutation of part {} started by some other replica, will wait for it and mutated merged part. Number of tries {}", + entry.new_part_name, + entry.num_tries); storage.watchZeroCopyLock(entry.new_part_name, disk); - LOG_DEBUG(log, "Mutation of part {} started by some other replica, will wait it and mutated merged part", entry.new_part_name); return PrepareResult{ .prepared_successfully = false, diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index b4748ee77ea..6ad77119016 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -174,7 +174,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() part_log_writer = prepare_result.part_log_writer; - /// Avoid resheduling, execute fetch here, in the same thread. + /// Avoid rescheduling, execute fetch here, in the same thread. if (!prepare_result.prepared_successfully) return execute_fetch(prepare_result.need_to_check_missing_part_in_fetch); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 07f46c07466..3264de850a0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1370,13 +1370,27 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (data_settings->allow_remote_fs_zero_copy_replication) { auto disks = storage.getDisks(); - bool only_s3_storage = true; + DiskPtr disk_with_zero_copy = nullptr; for (const auto & disk : disks) - if (!disk->supportZeroCopyReplication()) - only_s3_storage = false; + { + if (disk->supportZeroCopyReplication()) + { + disk_with_zero_copy = disk; + break; + } + } + /// Technically speaking if there are more than one disk that could store the part (a local hot + cloud cold) + /// It would be possible for the merge to happen concurrently with other replica if the other replica is doing + /// a merge using zero-copy and the cloud storage, and the local replica uses the local storage instead + /// The question is, is it worth keep retrying to do the merge over and over for the opportunity to do + /// double the work? Probably not + /// So what we do is that, even if hot merge could happen, check the zero copy lock anyway. + /// Keep in mind that for the zero copy lock check to happen (via existing_zero_copy_locks) we need to + /// have failed first because of it and added it via watchZeroCopyLock. Considering we've already tried to + /// use cloud storage and zero-copy replication, the most likely scenario is that we'll try again String replica_to_execute_merge; - if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge)) + if (disk_with_zero_copy && storage.checkZeroCopyLockExists(entry.new_part_name, disk_with_zero_copy, replica_to_execute_merge)) { constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after."; out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge); From 920887f315e108da3b385986dee329a28aed65fb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Jul 2023 16:43:59 +0000 Subject: [PATCH 447/473] Done --- .../test_replicated_merge_tree_encrypted_disk/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py b/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py index 05d7bbb7282..25d30eb9c82 100644 --- a/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py +++ b/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py @@ -67,6 +67,8 @@ def optimize_table(): def check_table(): expected = [[1, "str1"], [2, "str2"]] + node1.query("SYSTEM SYNC REPLICA tbl LIGHTWEIGHT") + node2.query("SYSTEM SYNC REPLICA tbl LIGHTWEIGHT") assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node1.query("CHECK TABLE tbl") == "1\n" From 6b3a508a23e62d5459ad2a19a3bfc91ca96ccb8f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Jul 2023 16:52:45 +0000 Subject: [PATCH 448/473] Done --- tests/queries/0_stateless/02122_parallel_formatting.lib | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02122_parallel_formatting.lib b/tests/queries/0_stateless/02122_parallel_formatting.lib index 56119012788..5175e004cc5 100755 --- a/tests/queries/0_stateless/02122_parallel_formatting.lib +++ b/tests/queries/0_stateless/02122_parallel_formatting.lib @@ -11,14 +11,14 @@ non_parallel_file=$CLICKHOUSE_TMP/$CLICKHOUSE_TEST_UNIQUE_NAME"_non_parallel" format=$1 echo $format-1 -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_write_statistics=0 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_write_statistics=0 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file echo $format-2 -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_write_statistics=0 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_write_statistics=0 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file From 500f1e6757b721ecc8733f5e8bf41c765a631918 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 20 Jul 2023 18:55:41 +0200 Subject: [PATCH 449/473] Follow up to #49698 --- .../PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index adbc95a2cf2..f2923b60bfd 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int TOO_MANY_PARTS; } MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( @@ -591,11 +590,8 @@ void MaterializedPostgreSQLConsumer::syncTables() } catch (DB::Exception & e) { - if (e.code() == ErrorCodes::TOO_MANY_PARTS) - { - /// Retry this buffer later. - storage_data.buffer.columns = result_rows.mutateColumns(); - } + /// Retry this buffer later. + storage_data.buffer.columns = result_rows.mutateColumns(); throw; } From 045ecdf71f544cafb4f5c3eda20ee6f9d593f614 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 20 Jul 2023 19:03:06 +0200 Subject: [PATCH 450/473] Update src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index f2923b60bfd..d01746ddf1b 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -588,7 +588,7 @@ void MaterializedPostgreSQLConsumer::syncTables() executor.execute(); } } - catch (DB::Exception & e) + catch (...) { /// Retry this buffer later. storage_data.buffer.columns = result_rows.mutateColumns(); From 09e6bbc0e2ac634cde658b9c53e599d124d0a3d8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 21:10:59 +0300 Subject: [PATCH 451/473] Update DataPartsExchange.cpp --- src/Storages/MergeTree/DataPartsExchange.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 23bbc1c7f9d..6a3bf2940e9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -353,8 +353,14 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) { /// It is important to include Outdated parts here because remote replicas cannot reliably /// determine the local state of the part, so queries for the parts in these states are completely normal. - auto part = data.getPartIfExists( - name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + MergeTreeData::DataPartPtr part; + + /// Ephemeral zero-copy lock may be lost for PreActive parts + bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; + if (zero_copy_enabled) + part = data.getPartIfExists(name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + else + part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (part) return part; From 24371c33bfd5037455cb025b057fb413ee1be396 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 21 Jul 2023 09:24:16 +0800 Subject: [PATCH 452/473] remove DelayedBlocksTask::finish --- src/Processors/Transforms/JoiningTransform.cpp | 3 ++- src/Processors/Transforms/JoiningTransform.h | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 5480fea27a4..4e7868ea1c2 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -375,7 +375,8 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() input.setNotNeeded(); } - if (task->finished) + // When delayed_blocks is nullptr, it means that all buckets have been joined. + if (!task->delayed_blocks) { input.close(); output.finish(); diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 5e7403dbbdb..a308af03662 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -115,7 +115,7 @@ class DelayedBlocksTask : public ChunkInfo { public: - explicit DelayedBlocksTask() : finished(true) {} + DelayedBlocksTask() = default; explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) : delayed_blocks(std::move(delayed_blocks_)) , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) @@ -125,7 +125,6 @@ public: IBlocksStreamPtr delayed_blocks = nullptr; JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; - bool finished = false; }; using DelayedBlocksTaskPtr = std::shared_ptr; From 51e81b37a4158e31b61c5ae8d993a2dee0ae16e3 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 7 Jul 2023 18:45:13 +0200 Subject: [PATCH 453/473] Move condtions with columns from PK to the end of PREWHERE chain --- .../MergeTree/MergeTreeWhereOptimizer.cpp | 31 +++++++++++++++++++ .../MergeTree/MergeTreeWhereOptimizer.h | 8 ++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 1620ba98d58..4ff58c1fc86 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -22,6 +22,33 @@ namespace DB /// This is used to assume that condition is likely to have good selectivity. static constexpr auto threshold = 2; +static NameToIndexMap fillNamesPositions(const Names & names) +{ + NameToIndexMap names_positions; + + for (size_t position = 0; position < names.size(); ++position) + { + const auto & name = names[position]; + names_positions[name] = position; + } + + return names_positions; +} + +/// Find minimal position of any of the column in primary key. +static Int64 findMinPosition(const NameSet & condition_table_columns, const NameToIndexMap & primary_key_positions) +{ + Int64 min_position = std::numeric_limits::max() - 1; + + for (const auto & column : condition_table_columns) + { + auto it = primary_key_positions.find(column); + if (it != primary_key_positions.end()) + min_position = std::min(min_position, static_cast(it->second)); + } + + return min_position; +} MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( std::unordered_map column_sizes_, @@ -35,6 +62,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , supported_columns{supported_columns_} , sorting_key_names{NameSet( metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())} + , primary_key_names_positions(fillNamesPositions(metadata_snapshot->getPrimaryKey().column_names)) , log{log_} , column_sizes{std::move(column_sizes_)} { @@ -234,6 +262,9 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); + /// Find min position in PK of any column that is used in this condition. + cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + res.emplace_back(std::move(cond)); } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 18555a72db1..8ab21471aeb 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -72,9 +72,14 @@ private: /// Does the condition presumably have good selectivity? bool good = false; + /// Does the condition contain primary key column? + /// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any + /// column in this condition because this condition have bigger chances to be already satisfied by PK analysis. + Int64 min_position_in_primary_key = std::numeric_limits::max() - 1; + auto tuple() const { - return std::make_tuple(!viable, !good, columns_size, table_columns.size()); + return std::make_tuple(!viable, !good, -min_position_in_primary_key, columns_size, table_columns.size()); } /// Is condition a better candidate for moving to PREWHERE? @@ -141,6 +146,7 @@ private: const Names queried_columns; const std::optional supported_columns; const NameSet sorting_key_names; + const NameToIndexMap primary_key_names_positions; Poco::Logger * log; std::unordered_map column_sizes; UInt64 total_size_of_queried_columns = 0; From fe7cff5c1cfee89d411ec79e7a3e7603ff831782 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 19 Jul 2023 22:45:04 +0200 Subject: [PATCH 454/473] Added move_primary_key_columns_to_end_of_prewhere setting --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 11 +++++++++-- src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 1 + 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..5a1f4b46223 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -129,6 +129,7 @@ class IColumn; M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \ M(Bool, move_all_conditions_to_prewhere, true, "Move all viable conditions from WHERE to PREWHERE", 0) \ M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ + M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 4ff58c1fc86..5efb7286685 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -88,6 +88,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons where_optimizer_context.context = context; where_optimizer_context.array_joined_names = determineArrayJoinedNames(select); where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = select.final(); RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); @@ -117,6 +118,7 @@ std::optional MergeTreeWhe where_optimizer_context.context = context; where_optimizer_context.array_joined_names = {}; where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = is_final; RPNBuilderTreeContext tree_context(context); @@ -262,8 +264,13 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); - /// Find min position in PK of any column that is used in this condition. - cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) + { + /// Consider all conditions good with this setting enabled. + cond.good = cond.viable; + /// Find min position in PK of any column that is used in this condition. + cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + } res.emplace_back(std::move(cond)); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 8ab21471aeb..fb5e84b67c6 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -96,6 +96,7 @@ private: ContextPtr context; NameSet array_joined_names; bool move_all_conditions_to_prewhere = false; + bool move_primary_key_columns_to_end_of_prewhere = false; bool is_final = false; }; From 9b0eb9cdd709418c3782ae2468693b294e81a0cd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:34:44 +0800 Subject: [PATCH 455/473] ignore ast opt when doing projection calc --- src/Storages/ProjectionsDescription.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 73fb279d51c..086355b6a79 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -298,6 +298,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState} + .ignoreASTOptimizations() .ignoreSettingConstraints()) .buildQueryPipeline(); builder.resize(1); From 696818b340d88667a214674f1df483b8c9e827d9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:35:18 +0800 Subject: [PATCH 456/473] Don't check monotonicity when analyze projections --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 4 ++-- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h | 2 +- .../Optimizations/optimizeUseAggregateProjection.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index c9cf46aaeca..1c18465e1e1 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -8,7 +8,7 @@ namespace DB { -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity) { using Parents = std::set; std::unordered_map inner_parents; @@ -182,7 +182,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG } } - if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) + if (!match.node && check_monotonicity && frame.node->function_base->hasInformationAboutMonotonicity()) { size_t num_const_args = 0; const ActionsDAG::Node * monotonic_child = nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h index dd689cba46b..223fc40e33f 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h @@ -39,5 +39,5 @@ struct MatchedTrees using Matches = std::unordered_map; }; -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag); +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity = true); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..cf88de19f03 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -287,7 +287,7 @@ ActionsDAGPtr analyzeAggregateProjection( { auto proj_index = buildDAGIndex(*info.before_aggregation); - MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag); + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag, false /* check_monotonicity */); // for (const auto & [node, match] : matches) // { From 60488e23912ba29ca0e75e2a39b1902517244e6b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:38:42 +0800 Subject: [PATCH 457/473] Add tests --- ...ggregate_projection_with_monotonic_key_expr.reference | 1 + ...1710_aggregate_projection_with_monotonic_key_expr.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql new file mode 100644 index 00000000000..c3109553f63 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS t0; + +CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); + +INSERT INTO t0(c0) VALUES (1); + +SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); + +DROP TABLE t0; From d7bb006c231c3960e9c9f7a6f07cd8ba299d6422 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jul 2023 18:02:02 +0000 Subject: [PATCH 458/473] Fix monotonic chain for read-in-order as well. --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 7 ++++++- ..._aggregate_projection_with_monotonic_key_expr.reference | 4 ++++ .../01710_aggregate_projection_with_monotonic_key_expr.sql | 7 +++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index 1c18465e1e1..787a106200a 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -75,7 +75,12 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG } /// A node from found match may be nullptr. /// It means that node is visited, but no match was found. - frame.mapped_children.push_back(it->second.node); + if (it->second.monotonicity) + /// Ignore a match with monotonicity. + frame.mapped_children.push_back(nullptr); + else + frame.mapped_children.push_back(it->second.node); + } if (frame.mapped_children.size() < frame.node->children.size()) diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index d00491fd7e5..06e9efbe839 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -1 +1,5 @@ 1 +1 +-1 +1 +-1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index c3109553f63..bed43ef6630 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); @@ -6,4 +7,10 @@ INSERT INTO t0(c0) VALUES (1); SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); +create table t1 (c0 Int32) engine = MergeTree order by sin(c0); +insert into t1 values (-1), (1); +select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; +select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; + DROP TABLE t0; +DROP TABLE t1; From ec223372848014b79990bc05318862b8f8e76212 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 10:54:26 +0800 Subject: [PATCH 459/473] Fix index analysis with indexHint as well --- src/Interpreters/ActionsVisitor.cpp | 8 ++++++++ ...regate_projection_with_monotonic_key_expr.reference | 1 + ...10_aggregate_projection_with_monotonic_key_expr.sql | 10 +++++++++- 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index efab11003f5..b769011e3d4 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -976,7 +976,15 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (node.name == "indexHint") { if (data.only_consts) + { + /// We need to collect constants inside `indexHint` for index analysis. + if (node.arguments) + { + for (const auto & arg : node.arguments->children) + visit(arg, data); + } return; + } /// Here we create a separate DAG for indexHint condition. /// It will be used only for index analysis. diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index 06e9efbe839..1c8b399e790 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -3,3 +3,4 @@ -1 1 -1 +0 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index bed43ef6630..5cd8ec87b4b 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -1,16 +1,24 @@ DROP TABLE IF EXISTS t0; DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); INSERT INTO t0(c0) VALUES (1); -SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); +SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0) SETTINGS optimize_use_implicit_projections = 1; create table t1 (c0 Int32) engine = MergeTree order by sin(c0); insert into t1 values (-1), (1); select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; +CREATE TABLE t2 (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; + +INSERT INTO t2 FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); + +SELECT count() FROM t2 WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; + DROP TABLE t0; DROP TABLE t1; +DROP TABLE t2; From 5560603321319243180a7d38b17be29e2f69cf30 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 10:56:37 +0800 Subject: [PATCH 460/473] optimize_use_implicit_projections=1 by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..b8207b142d3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -577,7 +577,7 @@ class IColumn; M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ - M(Bool, optimize_use_implicit_projections, false, "Automatically choose implicit projections to perform SELECT query", 0) \ + M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3e58750e1d2..2886cdd288d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,6 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.7", {{"optimize_use_implicit_projections", true, false, "Disable implicit projections due to unexpected results."}}}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, From 68e7583dbfaca500757ba0b8e3d3d859b89accfa Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 14:53:06 +0800 Subject: [PATCH 461/473] reorganize tests and add some comments --- .../Optimizations/optimizeUseAggregateProjection.cpp | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- ...aggregate_projection_with_monotonic_key_expr.reference | 1 - ...01710_aggregate_projection_with_monotonic_key_expr.sql | 7 ------- tests/queries/0_stateless/01739_index_hint.reference | 5 +++++ tests/queries/0_stateless/01739_index_hint.sql | 8 ++++++++ 6 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index cf88de19f03..e611bb5b2ef 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -497,6 +497,9 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); + // minmax_count_projection cannot be used used when there is no data to process, because + // it will produce incorrect result during constant aggregation. + // See https://github.com/ClickHouse/ClickHouse/issues/36728 if (block) { MinMaxProjectionCandidate minmax; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9c5e45aa488..6c1375ecc1d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7027,7 +7027,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg max_added_blocks.get(), query_context); - // minmax_count_projection should not be used when there is no data to process. + // minmax_count_projection cannot be used used when there is no data to process, because + // it will produce incorrect result during constant aggregation. + // See https://github.com/ClickHouse/ClickHouse/issues/36728 if (!query_info.minmax_count_projection_block) return; diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index 1c8b399e790..06e9efbe839 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -3,4 +3,3 @@ -1 1 -1 -0 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index 5cd8ec87b4b..51dafb07b91 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -13,12 +13,5 @@ insert into t1 values (-1), (1); select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; -CREATE TABLE t2 (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; - -INSERT INTO t2 FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); - -SELECT count() FROM t2 WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; - DROP TABLE t0; DROP TABLE t1; -DROP TABLE t2; diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 3a4b380de65..766dff8c7b0 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -33,3 +33,8 @@ insert into XXXX select number*60, 0 from numbers(100000); SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; 100000 drop table XXXX; +CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; +INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; +0 +drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index e1e66c630e1..77c2760535d 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -33,3 +33,11 @@ insert into XXXX select number*60, 0 from numbers(100000); SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; drop table XXXX; + +CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; + +INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); + +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; + +drop table XXXX; From 0c2ea94efeeb9a68448d44c1eba08ad3898ef99f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 09:57:32 +0200 Subject: [PATCH 462/473] Update test --- .../0_stateless/02156_storage_merge_prewhere.reference | 2 +- tests/queries/0_stateless/02156_storage_merge_prewhere.sql | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 30f9b1ab175..74ba452d783 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,6 +1,6 @@ SELECT count() FROM t_02156_merge1 -PREWHERE (k = 3) AND notEmpty(v) +PREWHERE notEmpty(v) AND (k = 3) 2 SELECT count() FROM t_02156_merge2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index b75d3fa22e5..83d88a68d9b 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -1,4 +1,5 @@ SET optimize_move_to_prewhere = 1; +SET enable_multiple_prewhere_read_steps = 1; DROP TABLE IF EXISTS t_02156_mt1; DROP TABLE IF EXISTS t_02156_mt2; @@ -8,8 +9,8 @@ DROP TABLE IF EXISTS t_02156_merge1; DROP TABLE IF EXISTS t_02156_merge2; DROP TABLE IF EXISTS t_02156_merge3; -CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; -CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; +CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; +CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; CREATE TABLE t_02156_log (k UInt32, v String) ENGINE = Log; CREATE TABLE t_02156_dist (k UInt32, v String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_mt1); From 3815a6aa62414fcd4e51090450a08bc368fd8c86 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Jul 2023 11:14:39 +0000 Subject: [PATCH 463/473] Make test_keeper_force_recovery more stable --- .../configs/enable_keeper1.xml | 2 ++ .../configs/enable_keeper2.xml | 2 ++ .../configs/enable_keeper3.xml | 2 ++ .../configs/enable_keeper4.xml | 2 ++ .../configs/enable_keeper5.xml | 2 ++ .../configs/enable_keeper6.xml | 2 ++ .../configs/enable_keeper7.xml | 2 ++ .../configs/enable_keeper8.xml | 2 ++ .../configs/recovered_keeper1.xml | 2 ++ .../configs/use_keeper.xml | 36 ------------------- .../test_keeper_force_recovery/test.py | 5 +-- .../configs/enable_keeper1.xml | 2 ++ .../configs/enable_keeper1_solo.xml | 2 ++ .../configs/enable_keeper2.xml | 2 ++ .../configs/enable_keeper3.xml | 2 ++ .../configs/use_keeper.xml | 16 --------- .../test.py | 5 +-- 17 files changed, 28 insertions(+), 60 deletions(-) delete mode 100644 tests/integration/test_keeper_force_recovery/configs/use_keeper.xml delete mode 100644 tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml index b7f9d1b058e..62109ee9092 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml index b773d59f259..2696c573180 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml @@ -1,5 +1,6 @@ + false 9181 2 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml index d4c2befc10f..fc0c0fd0300 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml @@ -1,5 +1,6 @@ + false 9181 3 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml index c039e709c9e..06f1c1d7195 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml @@ -1,5 +1,6 @@ + false 9181 4 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml index fb43b6524c8..5d3767ae969 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml @@ -1,5 +1,6 @@ + false 9181 5 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml index 430e662bf36..4d30822741a 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml @@ -1,5 +1,6 @@ + false 9181 6 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml index aa10774ad7d..b59141042ea 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml @@ -1,5 +1,6 @@ + false 9181 7 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml index 4f1c21a1084..711d70cb1ac 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml @@ -1,5 +1,6 @@ + false 9181 8 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml b/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml index eaf0f01afc9..abd4ef85bf7 100644 --- a/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml b/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml deleted file mode 100644 index f41e8c6e49c..00000000000 --- a/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - node4 - 9181 - - - node5 - 9181 - - - node6 - 9181 - - - node7 - 9181 - - - node8 - 9181 - - - diff --git a/tests/integration/test_keeper_force_recovery/test.py b/tests/integration/test_keeper_force_recovery/test.py index f7c3787b4d8..f630e5a422b 100644 --- a/tests/integration/test_keeper_force_recovery/test.py +++ b/tests/integration/test_keeper_force_recovery/test.py @@ -22,10 +22,7 @@ def get_nodes(): nodes.append( cluster.add_instance( f"node{i+1}", - main_configs=[ - f"configs/enable_keeper{i+1}.xml", - f"configs/use_keeper.xml", - ], + main_configs=[f"configs/enable_keeper{i+1}.xml"], stay_alive=True, ) ) diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml index 441c1bc185d..94e59128bd3 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml index f0cb887b062..6367b4b4c29 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml @@ -1,5 +1,6 @@ + false 1 9181 1 @@ -11,6 +12,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml index e2e2c1fd7db..548d12c2e0a 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml @@ -1,5 +1,6 @@ + false 9181 2 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml index e2ac0400d88..65f9675cbd6 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml @@ -1,5 +1,6 @@ + false 9181 3 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml deleted file mode 100644 index 384e984f210..00000000000 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - diff --git a/tests/integration/test_keeper_force_recovery_single_node/test.py b/tests/integration/test_keeper_force_recovery_single_node/test.py index 1c0d5e9a306..132c5488df6 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/test.py +++ b/tests/integration/test_keeper_force_recovery_single_node/test.py @@ -20,10 +20,7 @@ def get_nodes(): nodes.append( cluster.add_instance( f"node{i+1}", - main_configs=[ - f"configs/enable_keeper{i+1}.xml", - f"configs/use_keeper.xml", - ], + main_configs=[f"configs/enable_keeper{i+1}.xml"], stay_alive=True, ) ) From 8970a531453cbfd9916681bf658ba605db0129de Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 21 Jul 2023 11:38:46 +0000 Subject: [PATCH 464/473] Fix `countSubstrings()` hang with empty needle and a column haystack --- src/Functions/CountSubstringsImpl.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index de00e9397d6..1501e50afcf 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -49,6 +49,9 @@ struct CountSubstringsImpl /// FIXME: suboptimal memset(&res[0], 0, res.size() * sizeof(res[0])); + if (needle.empty()) + return; // Return all zeros + /// Current index in the array of strings. size_t i = 0; From 66c8066cd59a685ded94af69d2cc37469cb29d77 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 21 Jul 2023 11:45:44 +0000 Subject: [PATCH 465/473] add test --- tests/queries/0_stateless/01590_countSubstrings.reference | 2 ++ tests/queries/0_stateless/01590_countSubstrings.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference index 056cb4c53b6..95031cd3856 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.reference +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -7,6 +7,8 @@ empty 0 0 0 +0 +0 char 1 2 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index dd2122ed6ff..6d2d87b1260 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -12,6 +12,8 @@ select 'empty'; select countSubstrings('', '.'); select countSubstrings('', ''); select countSubstrings('.', ''); +select countSubstrings(toString(number), '') from numbers(1); +select countSubstrings('', toString(number)) from numbers(1); select 'char'; select countSubstrings('foobar.com', '.'); From 1fe8076b9422e09b78887b44a724cf1cae5d7fd8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 20 Jul 2023 12:38:57 +0200 Subject: [PATCH 466/473] Fix capabilities installed via systemd service (fixes netlink/IO priorities) CapabilityBoundingSet that contained in systemd unit before is about allowing to set some capabilities, not about granting them. To grant them you need to use AmbientCapabilities. And if you do not use 'clickhouse install' then: - IO priorities was unavailable (since they requires CAP_SYS_NICE) - For taskstats the procfs was used instead of netlink Not a big deal, but still. Here how it had been tested: $ systemd-run -p CapabilityBoundingSet=CAP_NET_ADMIN --shell root:/etc (master)# capsh --print Current: cap_net_admin=ep Bounding set =cap_net_admin Ambient set = $ systemd-run -p User=azat -p CapabilityBoundingSet=CAP_NET_ADMIN --shell azat:/etc$ capsh --print Current: = Bounding set =cap_net_admin Ambient set = $ systemd-run -p User=azat -p AmbientCapabilities=CAP_NET_ADMIN -p CapabilityBoundingSet=CAP_NET_ADMIN --shell azat:/etc$ capsh --print Current: cap_net_admin=eip Bounding set =cap_net_admin Ambient set =cap_net_admin Note, if you are running it under root (without changing user) you don't need to specify AmbientCapabilities additionally, because root has all capabilities by default and they had been inherited. Signed-off-by: Azat Khuzhin --- packages/clickhouse-server.service | 1 + 1 file changed, 1 insertion(+) diff --git a/packages/clickhouse-server.service b/packages/clickhouse-server.service index 7742d8b278a..42dc5bd380d 100644 --- a/packages/clickhouse-server.service +++ b/packages/clickhouse-server.service @@ -29,6 +29,7 @@ EnvironmentFile=-/etc/default/clickhouse LimitCORE=infinity LimitNOFILE=500000 CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE +AmbientCapabilities=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE [Install] # ClickHouse should not start from the rescue shell (rescue.target). From b45c2c939b974ea3306f9c2192f362d71a69c0e2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 21 Jul 2023 15:17:07 +0300 Subject: [PATCH 467/473] disable expression templates for time intervals (#52335) --- .../Impl/ConstantExpressionTemplate.cpp | 33 +++++++++++++++++++ .../Formats/Impl/ConstantExpressionTemplate.h | 2 ++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 5 +++ ...2830_insert_values_time_interval.reference | 4 +++ .../02830_insert_values_time_interval.sql | 25 ++++++++++++++ 5 files changed, 69 insertions(+) create mode 100644 tests/queries/0_stateless/02830_insert_values_time_interval.reference create mode 100644 tests/queries/0_stateless/02830_insert_values_time_interval.sql diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 5d438d47de6..06efe0a20aa 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -177,6 +177,14 @@ private: if (function.name == "lambda") return; + /// Parsing of INTERVALs is quite hacky. Expressions are rewritten during parsing like this: + /// "now() + interval 1 day" -> "now() + toIntervalDay(1)" + /// "select now() + INTERVAL '1 day 1 hour 1 minute'" -> "now() + (toIntervalDay(1), toIntervalHour(1), toIntervalMinute(1))" + /// so the AST is completely different from the original expression . + /// Avoid extracting these literals and simply compare tokens. It makes the template less flexible but much simpler. + if (function.name.starts_with("toInterval")) + return; + FunctionOverloadResolverPtr builder = FunctionFactory::instance().get(function.name, context); /// Do not replace literals which must be constant ColumnNumbers dont_visit_children = builder->getArgumentsThatAreAlwaysConstant(); @@ -350,6 +358,31 @@ ConstantExpressionTemplate::TemplateStructure::TemplateStructure(LiteralsInfo & } +String ConstantExpressionTemplate::TemplateStructure::dumpTemplate() const +{ + WriteBufferFromOwnString res; + + size_t cur_column = 0; + size_t cur_token = 0; + size_t num_columns = literals.columns(); + while (cur_column < num_columns) + { + size_t skip_tokens_until = token_after_literal_idx[cur_column]; + while (cur_token < skip_tokens_until) + res << quote << tokens[cur_token++] << ", "; + + const DataTypePtr & type = literals.getByPosition(cur_column).type; + res << type->getName() << ", "; + ++cur_column; + } + + while (cur_token < tokens.size()) + res << quote << tokens[cur_token++] << ", "; + + res << "eof"; + return res.str(); +} + size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTPtr & expression, const LiteralsInfo & replaced_literals, const DataTypePtr & result_column_type, diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index fbb3cbcd22a..71d0d0f7134 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -31,6 +31,8 @@ class ConstantExpressionTemplate : boost::noncopyable static size_t getTemplateHash(const ASTPtr & expression, const LiteralsInfo & replaced_literals, const DataTypePtr & result_column_type, bool null_as_default, const String & salt); + String dumpTemplate() const; + String result_column_name; std::vector tokens; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index d61e723fd75..3a65a6fe4ea 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -474,6 +475,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx context, &found_in_cache, delimiter); + + LOG_TEST(&Poco::Logger::get("ValuesBlockInputFormat"), "Will use an expression template to parse column {}: {}", + column_idx, structure->dumpTemplate()); + templates[column_idx].emplace(structure); if (found_in_cache) ++attempts_to_deduce_template_cached[column_idx]; diff --git a/tests/queries/0_stateless/02830_insert_values_time_interval.reference b/tests/queries/0_stateless/02830_insert_values_time_interval.reference new file mode 100644 index 00000000000..b5b57fbfbfe --- /dev/null +++ b/tests/queries/0_stateless/02830_insert_values_time_interval.reference @@ -0,0 +1,4 @@ +1 2023-07-21 22:54:02 +2 2023-07-21 21:53:01 +3 2023-07-21 21:53:01 +4 2023-07-20 21:54:02 diff --git a/tests/queries/0_stateless/02830_insert_values_time_interval.sql b/tests/queries/0_stateless/02830_insert_values_time_interval.sql new file mode 100644 index 00000000000..f5d5d8a4c04 --- /dev/null +++ b/tests/queries/0_stateless/02830_insert_values_time_interval.sql @@ -0,0 +1,25 @@ + +DROP TABLE IF EXISTS t1; + +CREATE TABLE t1 +( + c1 DateTime DEFAULT now() NOT NULL, + c2 DateTime DEFAULT now() NOT NULL, + c3 DateTime DEFAULT now() NOT NULL, + PRIMARY KEY(c1, c2, c3) +) ENGINE = MergeTree() +ORDER BY (c1, c2, c3); + +INSERT INTO t1 (c1,c2,c3) VALUES(now() + INTERVAL '1 day 1 hour 1 minute 1 second', now(), now()); + +DROP TABLE t1; + +CREATE TABLE t1 (n int, dt DateTime) ENGINE=Memory; + +SET input_format_values_interpret_expressions=0; +INSERT INTO t1 VALUES (1, toDateTime('2023-07-20 21:53:01') + INTERVAL '1 day 1 hour 1 minute 1 second'), (2, toDateTime('2023-07-20 21:53:01') + INTERVAL '1 day'); +INSERT INTO t1 VALUES (3, toDateTime('2023-07-20 21:53:01') + INTERVAL 1 DAY), (4, toDateTime('2023-07-20 21:53:01') + (toIntervalMinute(1), toIntervalSecond(1))); + +SELECT * FROM t1 ORDER BY n; + +DROP TABLE t1; From 8d4c840e2da0401787bafa2239907ff59160a003 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:15:35 +0200 Subject: [PATCH 468/473] Some more cases --- tests/queries/0_stateless/01590_countSubstrings.reference | 3 +++ tests/queries/0_stateless/01590_countSubstrings.sql | 3 +++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference index 95031cd3856..367b910e569 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.reference +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -9,6 +9,9 @@ empty 0 0 0 +0 +0 +0 char 1 2 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index 6d2d87b1260..b38cbb7d188 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -14,6 +14,9 @@ select countSubstrings('', ''); select countSubstrings('.', ''); select countSubstrings(toString(number), '') from numbers(1); select countSubstrings('', toString(number)) from numbers(1); +select countSubstrings('aaa', materialize('')); +select countSubstrings(materialize('aaa'), ''); +select countSubstrings(materialize('aaa'), materialize('')); select 'char'; select countSubstrings('foobar.com', '.'); From b5cf64466887e115656aab065848fb52784964ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 21 Jul 2023 15:23:04 +0200 Subject: [PATCH 469/473] Check projection metadata the same way we check ordinary metadata. (#52361) * Check projection metadata the same way we check ordinary metadata. * Allow aggregate projection to have empty PK --------- Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../02540_duplicate_primary_key.sql | 18 +----------------- .../02540_duplicate_primary_key2.reference | 1 - .../02540_duplicate_primary_key2.sql | 10 +--------- .../02816_check_projection_metadata.reference | 0 .../02816_check_projection_metadata.sql | 3 +++ 7 files changed, 13 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02816_check_projection_metadata.reference create mode 100644 tests/queries/0_stateless/02816_check_projection_metadata.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a820bacf9a3..34be8156e71 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -465,9 +465,10 @@ void MergeTreeData::checkProperties( const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, + bool allow_empty_sorting_key, ContextPtr local_context) const { - if (!new_metadata.sorting_key.definition_ast) + if (!new_metadata.sorting_key.definition_ast && !allow_empty_sorting_key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty"); KeyDescription new_sorting_key = new_metadata.sorting_key; @@ -580,6 +581,9 @@ void MergeTreeData::checkProperties( if (projections_names.find(projection.name) != projections_names.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection with name {} already exists", backQuote(projection.name)); + /// We cannot alter a projection so far. So here we do not try to find a projection in old metadata. + bool is_aggregate = projection.type == ProjectionDescription::Type::Aggregate; + checkProperties(*projection.metadata, *projection.metadata, attach, is_aggregate, local_context); projections_names.insert(projection.name); } } @@ -593,7 +597,7 @@ void MergeTreeData::setProperties( bool attach, ContextPtr local_context) { - checkProperties(new_metadata, old_metadata, attach, local_context); + checkProperties(new_metadata, old_metadata, attach, false, local_context); setInMemoryMetadata(new_metadata); } @@ -3286,7 +3290,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } - checkProperties(new_metadata, old_metadata, false, local_context); + checkProperties(new_metadata, old_metadata, false, false, local_context); checkTTLExpressions(new_metadata, old_metadata); if (!columns_to_check_conversion.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8b5b50b1841..28611d09386 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1229,7 +1229,7 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr) const; + void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, bool allow_empty_sorting_key, ContextPtr local_context) const; void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr); diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key.sql b/tests/queries/0_stateless/02540_duplicate_primary_key.sql index a084d76964b..6905c9d5133 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key.sql +++ b/tests/queries/0_stateless/02540_duplicate_primary_key.sql @@ -86,20 +86,4 @@ CREATE TABLE test ) ENGINE = MergeTree PARTITION BY toYYYYMM(coverage) -ORDER BY (coverage, situation_name, NAME_toe, NAME_cockroach); - -insert into test select * from generateRandom() limit 10; - -with dissonance as ( - Select cast(toStartOfInterval(coverage, INTERVAL 1 day) as Date) as flour, count() as regulation - from test - group by flour having flour >= toDate(now())-100 - ), -cheetah as ( - Select flour, regulation from dissonance - union distinct - Select toDate(now())-1, ifnull((select regulation from dissonance where flour = toDate(now())-1),0) as regulation -) -Select flour, regulation from cheetah order by flour with fill step 1 limit 100 format Null; - -drop table test; +ORDER BY (coverage, situation_name, NAME_toe, NAME_cockroach); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key2.reference b/tests/queries/0_stateless/02540_duplicate_primary_key2.reference index 08839f6bb29..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key2.reference +++ b/tests/queries/0_stateless/02540_duplicate_primary_key2.reference @@ -1 +0,0 @@ -200 diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key2.sql b/tests/queries/0_stateless/02540_duplicate_primary_key2.sql index d0f02a894f2..53800c95550 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key2.sql +++ b/tests/queries/0_stateless/02540_duplicate_primary_key2.sql @@ -88,12 +88,4 @@ CREATE TABLE test ) ENGINE = MergeTree PARTITION BY toYYYYMM(timestamp) -ORDER BY (xxxx17, xxxx14, xxxx16, toStartOfDay(timestamp), left(xxxx19, 10), timestamp); - -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'Airtel' as xxxx14 FROM generateRandom() LIMIT 100; -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'BSNL' as xxxx14 FROM generateRandom() LIMIT 100; -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'xxx' as xxxx14 FROM generateRandom() LIMIT 100; - -select sum(1) from test where toStartOfInterval(timestamp, INTERVAL 1 day) >= TIMESTAMP '2022-02-01 01:00:00' and xxxx14 in ('Airtel', 'BSNL') and xxxx1 = 1 GROUP BY xxxx16; - -drop table test; +ORDER BY (xxxx17, xxxx14, xxxx16, toStartOfDay(timestamp), left(xxxx19, 10), timestamp); -- { serverError BAD_ARGUMENTS} diff --git a/tests/queries/0_stateless/02816_check_projection_metadata.reference b/tests/queries/0_stateless/02816_check_projection_metadata.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02816_check_projection_metadata.sql b/tests/queries/0_stateless/02816_check_projection_metadata.sql new file mode 100644 index 00000000000..e7da043ad41 --- /dev/null +++ b/tests/queries/0_stateless/02816_check_projection_metadata.sql @@ -0,0 +1,3 @@ +create table kek (uuid FixedString(16), id int, ns String, dt DateTime64(6), projection null_pk (select * order by ns, 1, 4)) engine=MergeTree order by (id, dt, uuid); -- {serverError ILLEGAL_COLUMN } +-- this query could segfault or throw LOGICAL_ERROR previously, when we did not check projection PK +-- insert into kek select * from generageRandom(10000); From cb53d762eae54eef9411e2cf7548927f83fe187b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:23:24 +0200 Subject: [PATCH 470/473] Fix one more case --- src/Functions/CountSubstringsImpl.h | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 1501e50afcf..8ba9ee99de8 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -226,16 +226,19 @@ struct CountSubstringsImpl const char * needle_beg = reinterpret_cast(&needle_data[prev_needle_offset]); size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); - - const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); - const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); - - const UInt8 * pos; - while ((pos = searcher.search(beg, end)) < end) + if (needle_size > 0) { - ++res[i]; - beg = pos + needle_size; + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); + + const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); + const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); + + const UInt8 * pos; + while ((pos = searcher.search(beg, end)) < end) + { + ++res[i]; + beg = pos + needle_size; + } } } From 3acb6005f041051b7c00c48df5035843744a7e24 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 21 Jul 2023 17:08:01 +0200 Subject: [PATCH 471/473] Reduce the number of syscalls in FileCache::loadMetadata --- src/Interpreters/Cache/FileCache.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 91d1c63e832..42cc7b80a66 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -870,13 +870,12 @@ void FileCache::loadMetadata() } size_t total_size = 0; - for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; - key_prefix_it != fs::directory_iterator();) + for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; key_prefix_it != fs::directory_iterator(); + key_prefix_it++) { const fs::path key_prefix_directory = key_prefix_it->path(); - key_prefix_it++; - if (!fs::is_directory(key_prefix_directory)) + if (!key_prefix_it->is_directory()) { if (key_prefix_directory.filename() != "status") { @@ -887,19 +886,19 @@ void FileCache::loadMetadata() continue; } - if (fs::is_empty(key_prefix_directory)) + fs::directory_iterator key_it{key_prefix_directory}; + if (key_it == fs::directory_iterator{}) { LOG_DEBUG(log, "Removing empty key prefix directory: {}", key_prefix_directory.string()); fs::remove(key_prefix_directory); continue; } - for (fs::directory_iterator key_it{key_prefix_directory}; key_it != fs::directory_iterator();) + for (/* key_it already initialized to verify emptiness */; key_it != fs::directory_iterator(); key_it++) { const fs::path key_directory = key_it->path(); - ++key_it; - if (!fs::is_directory(key_directory)) + if (!key_it->is_directory()) { LOG_DEBUG( log, @@ -908,7 +907,7 @@ void FileCache::loadMetadata() continue; } - if (fs::is_empty(key_directory)) + if (fs::directory_iterator{key_directory} == fs::directory_iterator{}) { LOG_DEBUG(log, "Removing empty key directory: {}", key_directory.string()); fs::remove(key_directory); From c6ffc9f266f1bb8a667a3d5beff9bd47a288ef74 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 01:05:39 +0300 Subject: [PATCH 472/473] Update 02815_fix_not_found_constants_col_in_block.sql --- .../0_stateless/02815_fix_not_found_constants_col_in_block.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql index c56d59c72d6..fa784cf12e3 100644 --- a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql @@ -3,3 +3,4 @@ CREATE TABLE t0 (vkey UInt32, c0 Float32, primary key(c0)) engine = AggregatingM insert into t0 values (19000, 1); select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc, c_2_2 asc; select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc; +DROP TABLE t0; From c3c6560c9511163fc14eac2be8f89c30d3bcce1d Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 22 Jul 2023 08:44:19 +0000 Subject: [PATCH 473/473] Add 02815_range_dict_no_direct_join to analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..19b90a39800 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,4 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02815_range_dict_no_direct_join