From 383fc06761f81bee735ec22692a2d506ca78c01e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 9 May 2023 16:10:53 +0200 Subject: [PATCH 001/141] 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 002/141] 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 003/141] 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 004/141] 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 5c7f255078327538ab856b5e47bb8aec431a243f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 08:02:20 +0200 Subject: [PATCH 005/141] Add a test for #41727 --- .../02806_cte_block_cannot_be_empty.reference | 3 +++ .../02806_cte_block_cannot_be_empty.sql | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02806_cte_block_cannot_be_empty.reference create mode 100644 tests/queries/0_stateless/02806_cte_block_cannot_be_empty.sql diff --git a/tests/queries/0_stateless/02806_cte_block_cannot_be_empty.reference b/tests/queries/0_stateless/02806_cte_block_cannot_be_empty.reference new file mode 100644 index 00000000000..10bcae06939 --- /dev/null +++ b/tests/queries/0_stateless/02806_cte_block_cannot_be_empty.reference @@ -0,0 +1,3 @@ +{"dd":"2023-06-24 00:00:00"} +{"dd":"2023-06-24 00:00:00"} 2023-06-24 00:00:00 +{"result_date":"2023-08-24"} diff --git a/tests/queries/0_stateless/02806_cte_block_cannot_be_empty.sql b/tests/queries/0_stateless/02806_cte_block_cannot_be_empty.sql new file mode 100644 index 00000000000..688dc1017f7 --- /dev/null +++ b/tests/queries/0_stateless/02806_cte_block_cannot_be_empty.sql @@ -0,0 +1,18 @@ +with c as ( select 1 ID, toDate('2023-06-24') dt, 0 p ) select multiIf(t.ID = 1, formatRowNoNewline('JSONEachRow', dd), '') AS params from (select ID, case when p = 0 then toString(date_add(hour, p, dt)) else '2022-01-01' end as dd from c) t; +with c as ( select 1 ID, toDate('2023-06-24') dt, 0 p ) select multiIf(t.ID = 1, formatRowNoNewline('JSONEachRow', dd), '') AS params, dd from (select ID, case when p = 0 then toString(date_add(hour, p, dt)) else '2022-01-01' end as dd from c) t; + +select + if( + outer_table.condition_value = 1, + formatRowNoNewline('JSONEachRow', outer_table.result_date), + '' + ) as json +from ( + select + 1 as condition_value, + date_add(month, inner_table.offset, toDate('2023-06-24')) as result_date + from ( + select + 2 as offset + ) inner_table + ) outer_table; From 5b21a58df47ff89811d73e6412522e2d659c1c62 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 29 Jun 2023 17:12:55 +0000 Subject: [PATCH 006/141] Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join --- src/Interpreters/PredicateExpressionsOptimizer.cpp | 5 ++++- tests/queries/0_stateless/00976_asof_join_on.reference | 1 + tests/queries/0_stateless/00976_asof_join_on.sql | 5 +++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index 6606e64f689..e64ff34b11f 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -118,7 +118,10 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_e if (table_element->table_join && isLeft(table_element->table_join->as()->kind)) continue; /// Skip right table optimization - if (table_element->table_join && isFull(table_element->table_join->as()->kind)) + if (table_element->table_join && ( + isFull(table_element->table_join->as()->kind) + || table_element->table_join->as()->strictness == JoinStrictness::Asof + || table_element->table_join->as()->strictness == JoinStrictness::Anti)) break; /// Skip left and right table optimization is_rewrite_tables |= tryRewritePredicatesToTable(tables_element[table_pos], tables_predicates[table_pos], diff --git a/tests/queries/0_stateless/00976_asof_join_on.reference b/tests/queries/0_stateless/00976_asof_join_on.reference index 4d1b1273363..433d896426c 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.reference +++ b/tests/queries/0_stateless/00976_asof_join_on.reference @@ -33,3 +33,4 @@ 1 3 1 4 2 1 2 3 2 2 2 3 +1 2 1 2 diff --git a/tests/queries/0_stateless/00976_asof_join_on.sql b/tests/queries/0_stateless/00976_asof_join_on.sql index 8060fb86831..afa125a9271 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.sql +++ b/tests/queries/0_stateless/00976_asof_join_on.sql @@ -23,5 +23,10 @@ SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t != B.t; -- { serverError SELECT A.a, A.t, B.b, B.t FROM A ASOF JOIN B ON A.a == B.b AND A.t < B.t OR A.a == B.b + 1 ORDER BY (A.a, A.t); -- { serverError 48 } +SELECT A.a, A.t, B.b, B.t FROM A +ASOF INNER JOIN (SELECT * FROM B UNION ALL SELECT 1, 3) AS B ON B.t <= A.t AND A.a == B.b +WHERE B.t != 3 ORDER BY (A.a, A.t) +; + DROP TABLE A; DROP TABLE B; From f497ba88c2ea5a0088d955ceec78e271a8bf2e94 Mon Sep 17 00:00:00 2001 From: Alex Cheng Date: Fri, 30 Jun 2023 14:52:47 +0800 Subject: [PATCH 007/141] correct an exception message. --- src/Functions/nested.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/nested.cpp b/src/Functions/nested.cpp index 0a094176a55..679bb4f73d8 100644 --- a/src/Functions/nested.cpp +++ b/src/Functions/nested.cpp @@ -119,7 +119,7 @@ public: if (!lhs_array->hasEqualOffsets(*rhs_array)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument {} of function {} have different array offsets", + "The argument 2 and argument {} of function {} have different array offsets", i + 1, getName()); From fcffe2b5a22a29e4821072df5dc3f716d3308b95 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 30 Jun 2023 16:31:18 +0200 Subject: [PATCH 008/141] Increase mmap trashold to unattainable value --- src/Common/Allocator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 5a66ddb63a2..6779fee58e6 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -8,7 +8,7 @@ * See also: https://gcc.gnu.org/legacy-ml/gcc-help/2017-12/msg00021.html */ #ifdef NDEBUG - __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 64 * (1ULL << 20); + __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 64 * (1ULL << 32); #else /** * In debug build, use small mmap threshold to reproduce more memory From 34bf0284ad0b684e6ee2061bbb4d852d7a0ab79a Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Jun 2023 16:18:30 +0000 Subject: [PATCH 009/141] Add RowBinaryWithDefaults format --- docs/en/interfaces/formats.md | 18 ++++++ .../Formats/Impl/BinaryRowInputFormat.cpp | 56 ++++++++++++++----- .../Formats/Impl/BinaryRowInputFormat.h | 4 +- src/TableFunctions/TableFunctionFormat.cpp | 13 ++++- .../02810_row_binary_with_defaults.reference | 6 ++ .../02810_row_binary_with_defaults.sql | 7 +++ 6 files changed, 89 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/02810_row_binary_with_defaults.reference create mode 100644 tests/queries/0_stateless/02810_row_binary_with_defaults.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 378a1c46d93..3b7eab9a9d3 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -76,6 +76,7 @@ The supported formats are: | [RowBinary](#rowbinary) | ✔ | ✔ | | [RowBinaryWithNames](#rowbinarywithnamesandtypes) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | +| [RowBinaryWithDefaults](#rowbinarywithdefaults) | ✔ | ✔ | | [Native](#native) | ✔ | ✔ | | [Null](#null) | ✗ | ✔ | | [XML](#xml) | ✗ | ✔ | @@ -1514,6 +1515,23 @@ If setting [input_format_with_types_use_header](/docs/en/operations/settings/set the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. ::: +## RowBinaryWithDefaults {#rowbinarywithdefaults} + +Similar to [RowBinary](#rowbinary), but with an extra byte before each column that indicates if default value should be used. + +Examples: + +```sql +:) select * from format('RowBinaryWithDefaults', 'x UInt32 default 42, y UInt32', x'010001000000') + +┌──x─┬─y─┐ +│ 42 │ 1 │ +└────┴───┘ +``` + +For column `x` there is only one byte `01` that indicates that default value should be used and no other data after this byte is provided. +For column `y` data starts with byte `00` that indicates that column has actual value that should be read from the subsequent data `01000000`. + ## RowBinary format settings {#row-binary-format-settings} - [format_binary_max_string_size](/docs/en/operations/settings/settings-formats.md/#format_binary_max_string_size) - The maximum allowed size for String in RowBinary format. Default value - `1GiB`. diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index a4f779076eb..ac5da172210 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -13,7 +13,8 @@ namespace ErrorCodes extern const int CANNOT_SKIP_UNKNOWN_FIELD; } -BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) +template +BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) : RowInputFormatWithNamesAndTypes( header, in_, @@ -22,16 +23,17 @@ BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & heade with_names_, with_types_, format_settings_, - std::make_unique(in_, format_settings_)) + std::make_unique>(in_, format_settings_)) { } - -BinaryFormatReader::BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesReader(in_, format_settings_) +template +BinaryFormatReader::BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesReader(in_, format_settings_) { } -std::vector BinaryFormatReader::readHeaderRow() +template +std::vector BinaryFormatReader::readHeaderRow() { std::vector fields; String field; @@ -43,13 +45,15 @@ std::vector BinaryFormatReader::readHeaderRow() return fields; } -std::vector BinaryFormatReader::readNames() +template +std::vector BinaryFormatReader::readNames() { readVarUInt(read_columns, *in); return readHeaderRow(); } -std::vector BinaryFormatReader::readTypes() +template +std::vector BinaryFormatReader::readTypes() { auto types = readHeaderRow(); for (const auto & type_name : types) @@ -57,26 +61,40 @@ std::vector BinaryFormatReader::readTypes() return types; } -bool BinaryFormatReader::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/) +template +bool BinaryFormatReader::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/) { + if constexpr (with_defaults) + { + UInt8 is_default; + readBinary(is_default, *in); + if (is_default) + { + column.insertDefault(); + return false; + } + } serialization->deserializeBinary(column, *in, format_settings); return true; } -void BinaryFormatReader::skipHeaderRow() +template +void BinaryFormatReader::skipHeaderRow() { String tmp; for (size_t i = 0; i < read_columns; ++i) readStringBinary(tmp, *in); } -void BinaryFormatReader::skipNames() +template +void BinaryFormatReader::skipNames() { readVarUInt(read_columns, *in); skipHeaderRow(); } -void BinaryFormatReader::skipTypes() +template +void BinaryFormatReader::skipTypes() { if (read_columns == 0) { @@ -87,7 +105,8 @@ void BinaryFormatReader::skipTypes() skipHeaderRow(); } -void BinaryFormatReader::skipField(size_t file_column) +template +void BinaryFormatReader::skipField(size_t file_column) { if (file_column >= read_data_types.size()) throw Exception(ErrorCodes::CANNOT_SKIP_UNKNOWN_FIELD, @@ -111,12 +130,21 @@ void registerInputFormatRowBinary(FormatFactory & factory) const IRowInputFormat::Params & params, const FormatSettings & settings) { - return std::make_shared(buf, sample, params, with_names, with_types, settings); + return std::make_shared>(buf, sample, params, with_names, with_types, settings); }); }; registerWithNamesAndTypes("RowBinary", register_func); factory.registerFileExtension("bin", "RowBinary"); + + factory.registerInputFormat("RowBinaryWithDefaults", []( + ReadBuffer & buf, + const Block & sample, + const IRowInputFormat::Params & params, + const FormatSettings & settings) + { + return std::make_shared>(buf, sample, params, false, false, settings); + }); } void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory) @@ -125,6 +153,8 @@ void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory) { return std::make_shared(buf, settings); }); + + } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 3d3d80f1043..6f2042d1315 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -12,6 +12,7 @@ class ReadBuffer; /** A stream for inputting data in a binary line-by-line format. */ +template class BinaryRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: @@ -25,6 +26,7 @@ public: std::string getDiagnosticInfo() override { return {}; } }; +template class BinaryFormatReader final : public FormatWithNamesAndTypesReader { public: @@ -54,7 +56,7 @@ public: BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); private: - BinaryFormatReader reader; + BinaryFormatReader reader; }; } diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index f5aff4bd098..2a46f839bbe 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include @@ -72,7 +73,17 @@ Block TableFunctionFormat::parseData(ColumnsDescription columns, ContextPtr cont auto read_buf = std::make_unique(data); auto input_format = context->getInputFormat(format, *read_buf, block, context->getSettingsRef().max_block_size); - auto pipeline = std::make_unique(input_format); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + if (columns.hasDefaults()) + { + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, columns, *input_format, context); + }); + } + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto reader = std::make_unique(*pipeline); std::vector blocks; diff --git a/tests/queries/0_stateless/02810_row_binary_with_defaults.reference b/tests/queries/0_stateless/02810_row_binary_with_defaults.reference new file mode 100644 index 00000000000..5a556d1a6a9 --- /dev/null +++ b/tests/queries/0_stateless/02810_row_binary_with_defaults.reference @@ -0,0 +1,6 @@ +42 +1 +42 +1 +\N +[(42,42)] diff --git a/tests/queries/0_stateless/02810_row_binary_with_defaults.sql b/tests/queries/0_stateless/02810_row_binary_with_defaults.sql new file mode 100644 index 00000000000..73662352c0a --- /dev/null +++ b/tests/queries/0_stateless/02810_row_binary_with_defaults.sql @@ -0,0 +1,7 @@ +select * from format('RowBinaryWithDefaults', 'x UInt32 default 42', x'01'); +select * from format('RowBinaryWithDefaults', 'x UInt32 default 42', x'0001000000'); +select * from format('RowBinaryWithDefaults', 'x Nullable(UInt32) default 42', x'01'); +select * from format('RowBinaryWithDefaults', 'x Nullable(UInt32) default 42', x'000001000000'); +select * from format('RowBinaryWithDefaults', 'x Nullable(UInt32) default 42', x'0001'); +select * from format('RowBinaryWithDefaults', 'x Array(Tuple(UInt32, UInt32)) default [(42, 42)]', x'01'); + From 425875a7f203a19c500d3e8d8679f7a276315f12 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 3 Jul 2023 15:40:39 +0200 Subject: [PATCH 010/141] Remove mmap from Allocator --- src/Common/Allocator.cpp | 26 +--- src/Common/Allocator.h | 180 ++++------------------ src/Common/Allocator_fwd.h | 2 +- src/Common/CurrentMetrics.cpp | 2 - src/Common/HashTable/HashTableAllocator.h | 2 +- 5 files changed, 38 insertions(+), 174 deletions(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 6779fee58e6..769df70d71e 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -1,26 +1,4 @@ #include "Allocator.h" -/** Keep definition of this constant in cpp file; otherwise its value - * is inlined into allocator code making it impossible to override it - * in third-party code. - * - * Note: extern may seem redundant, but is actually needed due to bug in GCC. - * See also: https://gcc.gnu.org/legacy-ml/gcc-help/2017-12/msg00021.html - */ -#ifdef NDEBUG - __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 64 * (1ULL << 32); -#else - /** - * In debug build, use small mmap threshold to reproduce more memory - * stomping bugs. Along with ASLR it will hopefully detect more issues than - * ASan. The program may fail due to the limit on number of memory mappings. - * - * Not too small to avoid too quick exhaust of memory mappings. - */ - __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 16384; -#endif - -template class Allocator; -template class Allocator; -template class Allocator; -template class Allocator; +template class Allocator; +template class Allocator; diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index 5180fbdaa2d..1e77e988326 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -36,51 +36,26 @@ #include -/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS -#ifndef MAP_ANONYMOUS -#define MAP_ANONYMOUS MAP_ANON -#endif - -/** - * Many modern allocators (for example, tcmalloc) do not do a mremap for - * realloc, even in case of large enough chunks of memory. Although this allows - * you to increase performance and reduce memory consumption during realloc. - * To fix this, we do mremap manually if the chunk of memory is large enough. - * The threshold (64 MB) is chosen quite large, since changing the address - * space is very slow, especially in the case of a large number of threads. We - * expect that the set of operations mmap/something to do/mremap can only be - * performed about 1000 times per second. - * - * P.S. This is also required, because tcmalloc can not allocate a chunk of - * memory greater than 16 GB. - * - * P.P.S. Note that MMAP_THRESHOLD symbol is intentionally made weak. It allows - * to override it during linkage when using ClickHouse as a library in - * third-party applications which may already use own allocator doing mmaps - * in the implementation of alloc/realloc. - */ -extern const size_t MMAP_THRESHOLD; - static constexpr size_t MALLOC_MIN_ALIGNMENT = 8; -namespace CurrentMetrics -{ - extern const Metric MMappedAllocs; - extern const Metric MMappedAllocBytes; -} - namespace DB { + namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int CANNOT_ALLOCATE_MEMORY; - extern const int CANNOT_MUNMAP; - extern const int CANNOT_MREMAP; extern const int LOGICAL_ERROR; } + } +/** Previously there was a code which tried to use manual mmap and mremap (clickhouse_mremap.h) for large allocations/reallocations (64MB+). + * Most modern allocators (including jemalloc) don't use mremap, so the idea was to take advantage from mremap system call for large reallocs. + * Actually jemalloc had support for mremap, but it was intentionally removed from codebase https://github.com/jemalloc/jemalloc/commit/e2deab7a751c8080c2b2cdcfd7b11887332be1bb. + * Our performance tests also shows that without manual mmap/mremap/munmap clickhouse is overall faster for about 1-2% and up to 5-7x for some types of queries. + * That is why we don't do manuall mmap/mremap/munmap here and completely rely on jemalloc for allocations of any size. + */ + /** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena. * Also used in hash tables. * The interface is different from std::allocator @@ -88,10 +63,8 @@ namespace ErrorCodes * - passing the size into the `free` method; * - by the presence of the `alignment` argument; * - the possibility of zeroing memory (used in hash tables); - * - random hint address for mmap - * - mmap_threshold for using mmap less or more */ -template +template class Allocator { public: @@ -109,7 +82,7 @@ public: try { checkSize(size); - freeNoTrack(buf, size); + freeNoTrack(buf); CurrentMemoryTracker::free(size); } catch (...) @@ -132,49 +105,26 @@ public: /// nothing to do. /// BTW, it's not possible to change alignment while doing realloc. } - else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD - && alignment <= MALLOC_MIN_ALIGNMENT) + else if (alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. CurrentMemoryTracker::realloc(old_size, new_size); void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) - DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + { + DB::throwFromErrno( + fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } buf = new_buf; if constexpr (clear_memory) if (new_size > old_size) memset(reinterpret_cast(buf) + old_size, 0, new_size - old_size); } - else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD) - { - /// Resize mmap'd memory region. - CurrentMemoryTracker::realloc(old_size, new_size); - - // On apple and freebsd self-implemented mremap used (common/mremap.h) - buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, - PROT_READ | PROT_WRITE, mmap_flags, -1, 0); - if (MAP_FAILED == buf) - DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", - ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); - - /// No need for zero-fill, because mmap guarantees it. - } - else if (new_size < MMAP_THRESHOLD) - { - /// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once. - CurrentMemoryTracker::realloc(old_size, new_size); - - void * new_buf = allocNoTrack(new_size, alignment); - memcpy(new_buf, buf, std::min(old_size, new_size)); - freeNoTrack(buf, old_size); - buf = new_buf; - } else { /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. - void * new_buf = alloc(new_size, alignment); memcpy(new_buf, buf, std::min(old_size, new_size)); free(buf, old_size); @@ -192,83 +142,38 @@ protected: static constexpr bool clear_memory = clear_memory_; - // Freshly mmapped pages are copy-on-write references to a global zero page. - // On the first write, a page fault occurs, and an actual writable page is - // allocated. If we are going to use this memory soon, such as when resizing - // hash tables, it makes sense to pre-fault the pages by passing - // MAP_POPULATE to mmap(). This takes some time, but should be faster - // overall than having a hot loop interrupted by page faults. - // It is only supported on Linux. - static constexpr int mmap_flags = MAP_PRIVATE | MAP_ANONYMOUS -#if defined(OS_LINUX) - | (mmap_populate ? MAP_POPULATE : 0) -#endif - ; - private: void * allocNoTrack(size_t size, size_t alignment) { void * buf; - size_t mmap_min_alignment = ::getPageSize(); - - if (size >= MMAP_THRESHOLD) + if (alignment <= MALLOC_MIN_ALIGNMENT) { - if (alignment > mmap_min_alignment) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, - "Too large alignment {}: more than page size when allocating {}.", - ReadableSize(alignment), ReadableSize(size)); + if constexpr (clear_memory) + buf = ::calloc(size, 1); + else + buf = ::malloc(size); - buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE, - mmap_flags, -1, 0); - if (MAP_FAILED == buf) - DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - /// No need for zero-fill, because mmap guarantees it. - - CurrentMetrics::add(CurrentMetrics::MMappedAllocs); - CurrentMetrics::add(CurrentMetrics::MMappedAllocBytes, size); + if (nullptr == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); } else { - if (alignment <= MALLOC_MIN_ALIGNMENT) - { - if constexpr (clear_memory) - buf = ::calloc(size, 1); - else - buf = ::malloc(size); + buf = nullptr; + int res = posix_memalign(&buf, alignment, size); - if (nullptr == buf) - DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } - else - { - buf = nullptr; - int res = posix_memalign(&buf, alignment, size); + if (0 != res) + DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); - if (0 != res) - DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), - DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); - - if constexpr (clear_memory) - memset(buf, 0, size); - } + if constexpr (clear_memory) + memset(buf, 0, size); } return buf; } - void freeNoTrack(void * buf, size_t size) + void freeNoTrack(void * buf) { - if (size >= MMAP_THRESHOLD) - { - if (0 != munmap(buf, size)) - DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); - - CurrentMetrics::sub(CurrentMetrics::MMappedAllocs); - CurrentMetrics::sub(CurrentMetrics::MMappedAllocBytes, size); - } - else - { - ::free(buf); - } + ::free(buf); } void checkSize(size_t size) @@ -277,21 +182,6 @@ private: if (size >= 0x8000000000000000ULL) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size); } - -#ifndef NDEBUG - /// In debug builds, request mmap() at random addresses (a kind of ASLR), to - /// reproduce more memory stomping bugs. Note that Linux doesn't do it by - /// default. This may lead to worse TLB performance. - void * getMmapHint() - { - return reinterpret_cast(std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(thread_local_rng)); - } -#else - void * getMmapHint() - { - return nullptr; - } -#endif }; @@ -367,7 +257,5 @@ constexpr size_t allocatorInitialBytes; -extern template class Allocator; -extern template class Allocator; -extern template class Allocator; +extern template class Allocator; +extern template class Allocator; diff --git a/src/Common/Allocator_fwd.h b/src/Common/Allocator_fwd.h index a13a4398654..a96bc2a503b 100644 --- a/src/Common/Allocator_fwd.h +++ b/src/Common/Allocator_fwd.h @@ -3,7 +3,7 @@ * This file provides forward declarations for Allocator. */ -template +template class Allocator; template diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index f2ddb7a84c0..2f716cfb7ef 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -171,8 +171,6 @@ M(PartsInMemory, "In-memory parts.") \ M(MMappedFiles, "Total number of mmapped files.") \ M(MMappedFileBytes, "Sum size of mmapped file regions.") \ - M(MMappedAllocs, "Total number of mmapped allocations") \ - M(MMappedAllocBytes, "Sum bytes of mmapped allocations") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ M(PendingAsyncInsert, "Number of asynchronous inserts that are waiting for flush.") \ M(KafkaConsumers, "Number of active Kafka consumers") \ diff --git a/src/Common/HashTable/HashTableAllocator.h b/src/Common/HashTable/HashTableAllocator.h index 47e3fdfc4b6..8252265111d 100644 --- a/src/Common/HashTable/HashTableAllocator.h +++ b/src/Common/HashTable/HashTableAllocator.h @@ -8,7 +8,7 @@ * table, so it makes sense to pre-fault the pages so that page faults don't * interrupt the resize loop. Set the allocator parameter accordingly. */ -using HashTableAllocator = Allocator; +using HashTableAllocator = Allocator; template using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory; From 2f287703ddacb833c8cb03c497c65fd87e4f888e Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 3 Jul 2023 16:59:43 +0200 Subject: [PATCH 011/141] Fix test --- tests/queries/0_stateless/01778_mmap_cache_infra.reference | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/01778_mmap_cache_infra.reference b/tests/queries/0_stateless/01778_mmap_cache_infra.reference index ed365028ecc..0e82b277bc1 100644 --- a/tests/queries/0_stateless/01778_mmap_cache_infra.reference +++ b/tests/queries/0_stateless/01778_mmap_cache_infra.reference @@ -2,7 +2,5 @@ CreatedReadBufferMMap CreatedReadBufferMMapFailed MMappedFileCacheHits MMappedFileCacheMisses -MMappedAllocBytes -MMappedAllocs MMappedFileBytes MMappedFiles 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 012/141] 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 8b6376005a730b9ae461d3fe93a55e51cd494181 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 20 Apr 2023 13:26:02 +0000 Subject: [PATCH 013/141] "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 014/141] 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 015/141] 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 016/141] 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 017/141] 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 018/141] 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 019/141] 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 020/141] 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 021/141] 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 022/141] 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 cd3080428ea3da6a71169c929e959a0c3f9c5d5b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:58:01 +0000 Subject: [PATCH 023/141] 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 024/141] 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 025/141] 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 575de8d497673f7db12ccaa893921957d6d6725e Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 14 Jul 2023 07:04:17 +0200 Subject: [PATCH 026/141] 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 027/141] 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 028/141] 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 801d0955ececaa7141b3ed5a4210afab6eb19d1d Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jul 2023 12:08:40 +0300 Subject: [PATCH 029/141] 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 030/141] 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 4669951db550f4d70039c678ffacafa15b966f6c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 09:06:29 +0200 Subject: [PATCH 031/141] 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 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 032/141] 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 033/141] 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 034/141] 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 035/141] 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 036/141] 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 037/141] 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 038/141] 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 039/141] 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 e38a9de7c997f0f23456c978cdd991b9e579d6b3 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 17 Jul 2023 10:17:20 +0800 Subject: [PATCH 040/141] 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 041/141] 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 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 042/141] 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 043/141] 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 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 044/141] 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 018546a57d4553c44613c11aa3b0eb616461e60c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Jul 2023 17:39:50 +0200 Subject: [PATCH 045/141] 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 046/141] 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 9c666552883bff611f6c15a2e7bcf3af3e0127a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:01:58 +0200 Subject: [PATCH 047/141] 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 048/141] 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 049/141] 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 050/141] 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 051/141] 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 fff1ae73691bc3bbd409b1743a2c85d18412f868 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jul 2023 17:08:36 +0200 Subject: [PATCH 052/141] 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 56a5446af3958d37c0f2cbc8e879beb3876b8857 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 18 Jul 2023 10:28:51 +0800 Subject: [PATCH 053/141] 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 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 054/141] 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 055/141] 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 14ab1f2a98ad50ea08d8e089d4c426c353bd686b Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 12 Jul 2023 10:18:39 +0200 Subject: [PATCH 056/141] 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 057/141] 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 058/141] 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 059/141] 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 060/141] 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 061/141] 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 0cfd12aba46030b2147def929fada009a05eb07c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jul 2023 12:20:56 +0200 Subject: [PATCH 062/141] 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 063/141] 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 f163953b277b6ab1bc3d0aa0c68d8f7a65ce28c0 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 13:52:59 +0200 Subject: [PATCH 064/141] 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 065/141] 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 5282fc2c3450535b2fe2854e4cb1176faed3cba3 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:04:17 +0200 Subject: [PATCH 066/141] 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 067/141] 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 068/141] 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 069/141] 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 070/141] 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 680903e89bb6923f5b2fd49f6111d8506f5e5c16 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jul 2023 16:54:33 +0200 Subject: [PATCH 071/141] 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 072/141] 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 073/141] 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 a19a1001f063ce6d992ffc08d6d05d3ef7342b66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:28:22 +0200 Subject: [PATCH 074/141] 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 629e0e0269dc96f88f781eb8a0a711667d50c92b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 16:06:21 +0800 Subject: [PATCH 075/141] 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 a86baab88b4444d5bf34e529bb737817daa20096 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:49:30 +0000 Subject: [PATCH 076/141] 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 077/141] 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 078/141] 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 079/141] 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 080/141] 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 081/141] 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 082/141] 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 083/141] 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 084/141] 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 085/141] 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 086/141] 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 087/141] 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 088/141] 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 089/141] 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 090/141] 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 091/141] 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 70543e8ef9fe8523c5604d62fac3376da91c6d2c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 08:47:53 +0000 Subject: [PATCH 092/141] 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 093/141] 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 094/141] 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 095/141] 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 380b4ffe2be4107ae3965cba19c5b697e7108128 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 12:29:39 +0200 Subject: [PATCH 096/141] 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 2ebbbf0000ce7f5767d754b0aee777a4255ab7b3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 19:13:45 +0800 Subject: [PATCH 097/141] 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 7837559dbfdc194f28681dda808bc06b6609dd8b Mon Sep 17 00:00:00 2001 From: Song Liyong Date: Wed, 12 Jul 2023 17:13:04 +0200 Subject: [PATCH 098/141] 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 0c86df519ffa8921b6c546b304705366838dfa21 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Jul 2023 12:41:25 +0000 Subject: [PATCH 099/141] 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 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 100/141] 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 101/141] 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 777026e42e00311c4a751e8899fac407d6c8b874 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 19 Jul 2023 13:43:40 +0000 Subject: [PATCH 102/141] 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 103/141] 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 104/141] 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 105/141] 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 27ca367b2cb349391946c45d7e3d22fe6d543c42 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jul 2023 10:01:07 +0200 Subject: [PATCH 106/141] 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 7a0de384d498497fd026283a8232fcb8ed8ea5e6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 16:46:51 +0000 Subject: [PATCH 107/141] 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 108/141] 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 109/141] 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 110/141] 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 111/141] 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 112/141] 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 113/141] 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 114/141] 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 db1b53d1bb8ed6aa71f47010c81a7f3ebb0ae65d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:18:48 +0300 Subject: [PATCH 115/141] 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 116/141] 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 117/141] 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 118/141] 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 119/141] 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 120/141] 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 121/141] 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 122/141] 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 123/141] 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 124/141] 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 125/141] 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 126/141] 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 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 127/141] 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 128/141] 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 129/141] 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 130/141] 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 131/141] 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 132/141] 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 9b0eb9cdd709418c3782ae2468693b294e81a0cd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:34:44 +0800 Subject: [PATCH 133/141] 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 134/141] 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 135/141] 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 136/141] 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 137/141] 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 138/141] 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 139/141] 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 b45c2c939b974ea3306f9c2192f362d71a69c0e2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 21 Jul 2023 15:17:07 +0300 Subject: [PATCH 140/141] 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 b5cf64466887e115656aab065848fb52784964ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 21 Jul 2023 15:23:04 +0200 Subject: [PATCH 141/141] 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);