From 84ffd76853a7a3b4531b22002adaa8d386507b29 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Jan 2021 14:43:11 +0000 Subject: [PATCH 001/105] Initial table sync and replication pre-startup --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 77 ++++++ .../PostgreSQL/PostgreSQLReplicaConsumer.h | 37 +++ .../PostgreSQLReplicationHandler.cpp | 247 ++++++++++++++++++ .../PostgreSQL/PostgreSQLReplicationHandler.h | 74 ++++++ .../PostgreSQLReplicationSettings.cpp | 40 +++ .../PostgreSQLReplicationSettings.h | 21 ++ .../StorageMaterializePostgreSQL.cpp | 113 ++++++++ .../PostgreSQL/StorageMaterializePostgreSQL.h | 40 +++ src/Storages/registerStorages.cpp | 2 + 9 files changed, 651 insertions(+) create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h create mode 100644 src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp create mode 100644 src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp new file mode 100644 index 00000000000..017613500ad --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -0,0 +1,77 @@ +#include "PostgreSQLReplicaConsumer.h" + +#include +#include + +namespace DB +{ + +PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( + const std::string & table_name_, + const std::string & conn_str, + const std::string & replication_slot_name_, + const std::string & publication_name_, + const LSNPosition & start_lsn) + : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) + , replication_slot_name(replication_slot_name_) + , publication_name(publication_name_) + , table_name(table_name_) + , connection(std::make_shared(conn_str)) + , current_lsn(start_lsn) +{ + replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); +} + + +void PostgreSQLReplicaConsumer::run() +{ + auto options = fmt::format(" (\"proto_version\" '1', \"publication_names\" '{}')", publication_name); + startReplication(replication_slot_name, current_lsn.lsn, -1, options); +} + + +void PostgreSQLReplicaConsumer::startReplication( + const std::string & slot_name, const std::string start_lsn, const int64_t /* timeline */, const std::string & plugin_args) +{ + std::string query_str = fmt::format("START_REPLICATION SLOT {} LOGICAL {}", slot_name, start_lsn); + + if (!plugin_args.empty()) + query_str += plugin_args; + + auto tx = std::make_unique(*replication_connection->conn()); + tx->exec(query_str); + + //pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); + //pqxx::result result{tx->exec(query_str)}; + //pqxx::row row{result[0]}; + //for (auto res : row) + //{ + // if (std::size(res)) + // LOG_TRACE(log, "GOT {}", res.as()); + // else + // LOG_TRACE(log, "GOT NULL"); + //} + + // while (true) + // { + // const std::vector * row{stream.read_row()}; + + // if (!row) + // { + // LOG_TRACE(log, "STREAM REPLICATION END"); + // stream.complete(); + // tx->commit(); + // break; + // } + // LOG_TRACE(log, "STARTED REPLICATION. GOT ROW SIZE", row->size()); + + // for (const auto idx : ext::range(0, row->size())) + // { + // auto current = (*row)[idx]; + // LOG_TRACE(log, "Started replication. GOT: {}", current); + // } + + //} +} + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h new file mode 100644 index 00000000000..800f444765a --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include "PostgreSQLReplicationHandler.h" +#include "pqxx/pqxx" + +namespace DB +{ + +class PostgreSQLReplicaConsumer +{ +public: + PostgreSQLReplicaConsumer( + const std::string & table_name_, + const std::string & conn_str_, + const std::string & replication_slot_name_, + const std::string & publication_name_, + const LSNPosition & start_lsn); + + void run(); + +private: + void startReplication( + const std::string & slot_name, const std::string start_lsn, const int64_t timeline, const std::string & plugin_args); + + Poco::Logger * log; + const std::string replication_slot_name; + const std::string publication_name; + + const std::string table_name; + PostgreSQLConnectionPtr connection, replication_connection; + + LSNPosition current_lsn; +}; + +} + diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp new file mode 100644 index 00000000000..7fbcf1d9ff2 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -0,0 +1,247 @@ +#include "PostgreSQLReplicationHandler.h" +#include "PostgreSQLReplicaConsumer.h" + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TABLE; + extern const int LOGICAL_ERROR; +} + +PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( + const std::string & database_name_, + const std::string & table_name_, + const std::string & conn_str, + const std::string & replication_slot_, + const std::string & publication_name_) + : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) + , database_name(database_name_) + , table_name(table_name_) + , replication_slot(replication_slot_) + , publication_name(publication_name_) + , connection(std::make_shared(conn_str)) +{ + /// Create a replication connection, through which it is possible to execute only commands from streaming replication protocol + /// interface. Passing 'database' as the value instructs walsender to connect to the database specified in the dbname parameter, + /// which will allow the connection to be used for logical replication from that database. + replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); + + /// Used commands require a specific transaction isolation mode. + replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); + + /// Non temporary replication slot should be the same at restart. + if (replication_slot.empty()) + replication_slot = fmt::format("{}_{}_ch_replication_slot", database_name, table_name); + + /// Temporary replication slot is used to determine a start lsn position and to acquire a snapshot for initial table synchronization. + temp_replication_slot = replication_slot + "_temp"; +} + + +void PostgreSQLReplicationHandler::startup() +{ + tx = std::make_shared(*connection->conn()); + if (publication_name.empty()) + { + publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); + + if (!isPublicationExist()) + createPublication(); + } + else if (!isPublicationExist()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Publication name '{}' is spesified in table arguments, but it does not exist", publication_name); + } + tx->commit(); + + startReplication(); +} + + +bool PostgreSQLReplicationHandler::isPublicationExist() +{ + std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); + pqxx::result result{tx->exec(query_str)}; + bool publication_exists = (result[0][0].as() == "t"); + + if (publication_exists) + LOG_TRACE(log, "Publication {} already exists. Using existing version", publication_name); + + return publication_exists; +} + + +void PostgreSQLReplicationHandler::createPublication() +{ + /* * It is also important that change replica identity for this table to be able to receive old values of updated rows: + * ALTER TABLE pgbench_accounts REPLICA IDENTITY FULL; + * * TRUNCATE and DDL are not included in PUBLICATION. + * * 'ONLY' means just a table, without descendants. + */ + std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, table_name); + try + { + tx->exec(query_str); + LOG_TRACE(log, "Created publication {}", publication_name); + } + catch (pqxx::undefined_table const &) + { + throw Exception(fmt::format("PostgreSQL table {}.{} does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); + } +} + + +void PostgreSQLReplicationHandler::startReplication() +{ + auto ntx = std::make_shared(*replication_connection->conn()); + + /// But it should not actually exist. May exist if failed to drop it before. + if (isReplicationSlotExist(ntx, temp_replication_slot)) + dropReplicationSlot(ntx, temp_replication_slot, true); + + std::string snapshot_name; + LSNPosition start_lsn; + + createTempReplicationSlot(ntx, start_lsn, snapshot_name); + + loadFromSnapshot(snapshot_name); + + /// Do not need this replication slot anymore (snapshot loaded and start lsn determined, will continue replication protocol + /// with another slot, which should be the same at restart (and reused) to minimize memory usage) + /// Non temporary replication slot should be deleted with drop table only. + LOG_DEBUG(log, "Dropping temporaty replication slot"); + dropReplicationSlot(ntx, temp_replication_slot, true); + + if (!isReplicationSlotExist(ntx, replication_slot)) + createReplicationSlot(ntx); + + PostgreSQLReplicaConsumer consumer( + table_name, + connection->conn_str(), + replication_slot, + publication_name, + start_lsn); + + LOG_DEBUG(log, "Commiting replication transaction"); + ntx->commit(); + + consumer.run(); +} + + +bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name) +{ + std::string query_str = fmt::format("SELECT active, restart_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); + pqxx::result result{ntx->exec(query_str)}; + + /// Replication slot does not exist + if (result.empty()) + return false; + + bool is_active = result[0][0].as(); + LOG_TRACE(log, "Replication slot {} already exists (active: {}). Restart lsn position is {}", + slot_name, is_active, result[0][0].as()); + + return true; +} + + +void PostgreSQLReplicationHandler::createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name) +{ + std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} TEMPORARY LOGICAL pgoutput EXPORT_SNAPSHOT", temp_replication_slot); + try + { + pqxx::result result{ntx->exec(query_str)}; + start_lsn.lsn = result[0][1].as(); + snapshot_name = result[0][2].as(); + LOG_TRACE(log, "Created temporary replication slot: {}, start lsn: {}, snapshot: {}", + temp_replication_slot, start_lsn.lsn, snapshot_name); + } + catch (Exception & e) + { + e.addMessage("while creating PostgreSQL replication slot {}", temp_replication_slot); + throw; + } +} + + +void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx) +{ + std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput", replication_slot); + try + { + pqxx::result result{ntx->exec(query_str)}; + LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, result[0][1].as()); + } + catch (Exception & e) + { + e.addMessage("while creating PostgreSQL replication slot {}", replication_slot); + throw; + } +} + + +void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, std::string & slot_name, bool use_replication_api) +{ + if (use_replication_api) + { + std::string query_str = fmt::format("DROP_REPLICATION_SLOT {}", slot_name); + ntx->exec(query_str); + } + else + { + pqxx::work work(*connection->conn()); + std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); + work.exec(query_str); + work.commit(); + } +} + + +/// Only used when MaterializePostgreSQL table is dropped. +void PostgreSQLReplicationHandler::checkAndDropReplicationSlot() +{ + auto ntx = std::make_shared(*replication_connection->conn()); + if (isReplicationSlotExist(ntx, replication_slot)) + dropReplicationSlot(ntx, replication_slot, false); + ntx->commit(); +} + + +void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) +{ + auto stx = std::make_unique(*connection->conn()); + /// Required to execute the following command. + stx->set_variable("transaction_isolation", "'repeatable read'"); + + std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); + stx->exec(query_str); + + LOG_DEBUG(log, "Created transaction snapshot"); + query_str = fmt::format("SELECT * FROM {}", table_name); + pqxx::result result{stx->exec(query_str)}; + if (!result.empty()) + { + pqxx::row row{result[0]}; + for (auto res : row) + { + if (std::size(res)) + LOG_TRACE(log, "GOT {}", res.as()); + else + LOG_TRACE(log, "GOT NULL"); + } + } + LOG_DEBUG(log, "Done loading from snapshot"); + stx->commit(); +} + + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h new file mode 100644 index 00000000000..06379f2ad4a --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -0,0 +1,74 @@ +#pragma once + +#include +#include +#include "pqxx/pqxx" + + +/* Implementation of logical streaming replication protocol: https://www.postgresql.org/docs/10/protocol-logical-replication.html. + */ + +namespace DB +{ + +struct LSNPosition +{ + std::string lsn; + + uint64_t getValue() + { + uint64_t upper_half, lower_half, result; + std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); + result = (upper_half << 32) + lower_half; + LOG_DEBUG(&Poco::Logger::get("LSNParsing"), + "Created replication slot. upper half: {}, lower_half: {}, start lsn: {}", + upper_half, lower_half, result); + return result; + } +}; + + +class PostgreSQLReplicationHandler +{ +public: + friend class PGReplicaLSN; + PostgreSQLReplicationHandler( + const std::string & database_name_, + const std::string & table_name_, + const std::string & conn_str_, + const std::string & replication_slot_name_, + const std::string & publication_name_); + + void startup(); + void checkAndDropReplicationSlot(); + +private: + using NontransactionPtr = std::shared_ptr; + + bool isPublicationExist(); + void createPublication(); + + bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); + void createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name); + void createReplicationSlot(NontransactionPtr ntx); + void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name, bool use_replication_api); + + void startReplication(); + void loadFromSnapshot(std::string & snapshot_name); + + Poco::Logger * log; + const std::string database_name, table_name; + + std::string replication_slot, publication_name; + std::string temp_replication_slot; + + PostgreSQLConnectionPtr connection; + PostgreSQLConnectionPtr replication_connection; + std::shared_ptr tx; + + //LSNPosition start_lsn, final_lsn; +}; + + +} + diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp new file mode 100644 index 00000000000..98173f7ca07 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp @@ -0,0 +1,40 @@ +#include "PostgreSQLReplicationSettings.h" +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) + +void MaterializePostgreSQLSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h new file mode 100644 index 00000000000..c85b3572356 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + class ASTStorage; + + +#define LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS(M) \ + M(String, postgresql_replication_slot_name, "", "PostgreSQL replication slot name.", 0) \ + M(String, postgresql_publication_name, "", "PostgreSQL publication name.", 0) \ + +DECLARE_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) + +struct MaterializePostgreSQLSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp new file mode 100644 index 00000000000..385eb8a8706 --- /dev/null +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -0,0 +1,113 @@ +#include "StorageMaterializePostgreSQL.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "PostgreSQLReplicationSettings.h" +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( + const StorageID & table_id_, + const String & remote_table_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const Context & context_, + std::shared_ptr replication_handler_) + : IStorage(table_id_) + , remote_table_name(remote_table_name_) + , global_context(context_) + , replication_handler(std::move(replication_handler_)) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); + +} + + +void StorageMaterializePostgreSQL::startup() +{ + replication_handler->startup(); +} + + +void StorageMaterializePostgreSQL::shutdown() +{ + //replication_handler->dropReplicationSlot(); +} + + +void registerStorageMaterializePostgreSQL(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + bool has_settings = args.storage_def->settings; + auto postgresql_replication_settings = std::make_unique(); + + if (has_settings) + postgresql_replication_settings->loadFromQuery(*args.storage_def); + + if (engine_args.size() != 5) + throw Exception("Storage MaterializePostgreSQL requires 5 parameters: " + "PostgreSQL('host:port', 'database', 'table', 'username', 'password'", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context); + + auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 5432); + const String & remote_table = engine_args[2]->as().value.safeGet(); + const String & remote_database = engine_args[1]->as().value.safeGet(); + + String connection_str; + connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + remote_database, + parsed_host_port.first, std::to_string(parsed_host_port.second), + engine_args[3]->as().value.safeGet(), + engine_args[4]->as().value.safeGet()); + + auto global_context(args.context.getGlobalContext()); + auto replication_slot_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_replication_slot_name.value); + auto publication_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_publication_name.value); + + PostgreSQLReplicationHandler replication_handler(remote_database, remote_table, connection_str, replication_slot_name, publication_name); + + return StorageMaterializePostgreSQL::create( + args.table_id, remote_table, args.columns, args.constraints, global_context, + std::make_shared(replication_handler)); + }; + + factory.registerStorage( + "MaterializePostgreSQL", + creator_fn, + StorageFactory::StorageFeatures{ .supports_settings = true, .source_access_type = AccessType::POSTGRES, + }); +} + +} + diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h new file mode 100644 index 00000000000..ef0eb4e75cd --- /dev/null +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -0,0 +1,40 @@ +#pragma once + +#include "config_core.h" + +#include +#include +#include +#include "PostgreSQLReplicationHandler.h" +#include "pqxx/pqxx" + +namespace DB +{ + +class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; + +public: + StorageMaterializePostgreSQL( + const StorageID & table_id_, + const String & remote_table_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const Context & context_, + std::shared_ptr replication_handler_); + + String getName() const override { return "MaterializePostgreSQL"; } + + void startup() override; + void shutdown() override; + +private: + String remote_table_name; + Context global_context; + + std::shared_ptr replication_handler; +}; + +} + diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 0022ee6bd4f..bd32de1c315 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -60,6 +60,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory); #if USE_LIBPQXX void registerStoragePostgreSQL(StorageFactory & factory); +void registerStorageMaterializePostgreSQL(StorageFactory & factory); #endif void registerStorages() @@ -117,6 +118,7 @@ void registerStorages() #if USE_LIBPQXX registerStoragePostgreSQL(factory); + registerStorageMaterializePostgreSQL(factory); #endif } From a1bcc5fb3987e8473a6d115aa33138fa34ea2873 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Jan 2021 15:29:22 +0000 Subject: [PATCH 002/105] Decode replication messages --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 235 ++++++++++++++---- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 11 +- .../PostgreSQLReplicationHandler.cpp | 24 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- 4 files changed, 217 insertions(+), 55 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 017613500ad..5010f574555 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -2,9 +2,19 @@ #include #include +#include +#include +#include +#include +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( const std::string & table_name_, @@ -23,55 +33,194 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( } +void PostgreSQLReplicaConsumer::readString(const char * message, size_t & pos, size_t size, String & result) +{ + assert(size > pos + 2); + char current = unhex2(message + pos); + pos += 2; + while (pos < size && current != '\0') + { + result += current; + current = unhex2(message + pos); + pos += 2; + } +} + + +Int32 PostgreSQLReplicaConsumer::readInt32(const char * message, size_t & pos) +{ + assert(size > pos + 8); + Int32 result = (UInt32(unhex2(message + pos)) << 24) + | (UInt32(unhex2(message + pos + 2)) << 16) + | (UInt32(unhex2(message + pos + 4)) << 8) + | (UInt32(unhex2(message + pos + 6))); + pos += 8; + return result; +} + + +Int16 PostgreSQLReplicaConsumer::readInt16(const char * message, size_t & pos) +{ + assert(size > pos + 4); + Int16 result = (UInt32(unhex2(message + pos)) << 8) + | (UInt32(unhex2(message + pos + 2))); + pos += 4; + return result; +} + + +Int8 PostgreSQLReplicaConsumer::readInt8(const char * message, size_t & pos) +{ + assert(size > pos + 2); + Int8 result = unhex2(message + pos); + pos += 2; + return result; +} + + +Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos) +{ + assert(size > pos + 16); + Int64 result = (UInt64(unhex4(message + pos)) << 48) + | (UInt64(unhex4(message + pos + 4)) << 32) + | (UInt64(unhex4(message + pos + 8)) << 16) + | (UInt64(unhex4(message + pos + 12))); + pos += 16; + return result; +} + + +void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos, size_t /* size */) +{ + Int16 num_columns = readInt16(message, pos); + /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data + LOG_DEBUG(log, "num_columns {}", num_columns); + for (int k = 0; k < num_columns; ++k) + { + char identifier = readInt8(message, pos); + Int32 col_len = readInt32(message, pos); + String result; + for (int i = 0; i < col_len; ++i) + { + result += readInt8(message, pos); + } + LOG_DEBUG(log, "identifier {}, col_len {}, result {}", identifier, col_len, result); + } + //readString(message, pos, size, result); +} + + +void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replication_message, size_t size) +{ + /// Skip '\x' + size_t pos = 2; + char type = readInt8(replication_message, pos); + + LOG_TRACE(log, "TYPE: {}", type); + switch (type) + { + case 'B': // Begin + { + Int64 transaction_end_lsn = readInt64(replication_message, pos); + Int64 transaction_commit_timestamp = readInt64(replication_message, pos); + LOG_DEBUG(log, "transaction lsn {}, transaction commit timespamp {}", + transaction_end_lsn, transaction_commit_timestamp); + break; + } + case 'C': // Commit + { + readInt8(replication_message, pos); + Int64 commit_lsn = readInt64(replication_message, pos); + Int64 transaction_end_lsn = readInt64(replication_message, pos); + /// Since postgres epoch + Int64 transaction_commit_timestamp = readInt64(replication_message, pos); + LOG_DEBUG(log, "commit lsn {}, transaction lsn {}, transaction commit timestamp {}", + commit_lsn, transaction_end_lsn, transaction_commit_timestamp); + break; + } + case 'O': // Origin + break; + case 'R': // Relation + { + Int32 relation_id = readInt32(replication_message, pos); + String relation_namespace, relation_name; + readString(replication_message, pos, size, relation_namespace); + readString(replication_message, pos, size, relation_name); + Int8 replica_identity = readInt8(replication_message, pos); + Int16 num_columns = readInt16(replication_message, pos); + + LOG_DEBUG(log, + "Replication message type 'R', relation_id: {}, namespace: {}, relation name {}, replica identity {}, columns number {}", + relation_id, relation_namespace, relation_name, replica_identity, num_columns); + + Int8 key; + Int32 data_type_id, type_modifier; + for (uint16_t i = 0; i < num_columns; ++i) + { + String column_name; + key = readInt8(replication_message, pos); + readString(replication_message, pos, size, column_name); + data_type_id = readInt32(replication_message, pos); + type_modifier = readInt32(replication_message, pos); + LOG_DEBUG(log, "Key {}, column name {}, data type id {}, type modifier {}", key, column_name, data_type_id, type_modifier); + } + + break; + } + case 'Y': // Type + break; + case 'I': // Insert + { + Int32 relation_id = readInt32(replication_message, pos); + Int8 new_tuple = readInt8(replication_message, pos); + LOG_DEBUG(log, "relationID {}, newTuple {}", relation_id, new_tuple); + readTupleData(replication_message, pos, size); + break; + } + case 'U': // Update + break; + case 'D': // Delete + break; + case 'T': // Truncate + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected byte1 value {} while parsing replication message", type); + } +} + + void PostgreSQLReplicaConsumer::run() { - auto options = fmt::format(" (\"proto_version\" '1', \"publication_names\" '{}')", publication_name); - startReplication(replication_slot_name, current_lsn.lsn, -1, options); -} - - -void PostgreSQLReplicaConsumer::startReplication( - const std::string & slot_name, const std::string start_lsn, const int64_t /* timeline */, const std::string & plugin_args) -{ - std::string query_str = fmt::format("START_REPLICATION SLOT {} LOGICAL {}", slot_name, start_lsn); - - if (!plugin_args.empty()) - query_str += plugin_args; - auto tx = std::make_unique(*replication_connection->conn()); - tx->exec(query_str); + /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. + std::string query_str = fmt::format( + "select data FROM pg_logical_slot_peek_binary_changes(" + "'{}', NULL, NULL, 'publication_names', '{}', 'proto_version', '1')", + replication_slot_name, publication_name); + pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); - //pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); - //pqxx::result result{tx->exec(query_str)}; - //pqxx::row row{result[0]}; - //for (auto res : row) - //{ - // if (std::size(res)) - // LOG_TRACE(log, "GOT {}", res.as()); - // else - // LOG_TRACE(log, "GOT NULL"); - //} + while (true) + { + const std::vector * row{stream.read_row()}; - // while (true) - // { - // const std::vector * row{stream.read_row()}; + if (!row) + { + LOG_TRACE(log, "STREAM REPLICATION END"); + stream.complete(); + tx->commit(); + break; + } - // if (!row) - // { - // LOG_TRACE(log, "STREAM REPLICATION END"); - // stream.complete(); - // tx->commit(); - // break; - // } - // LOG_TRACE(log, "STARTED REPLICATION. GOT ROW SIZE", row->size()); - - // for (const auto idx : ext::range(0, row->size())) - // { - // auto current = (*row)[idx]; - // LOG_TRACE(log, "Started replication. GOT: {}", current); - // } - - //} + for (const auto idx : ext::range(0, row->size())) + { + LOG_TRACE(log, "Replication message: {}", (*row)[idx]); + decodeReplicationMessage((*row)[idx].c_str(), (*row)[idx].size()); + } + } } + } + + diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 800f444765a..e2833676412 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -1,6 +1,6 @@ #pragma once -#include +#include "PostgreSQLConnection.h" #include "PostgreSQLReplicationHandler.h" #include "pqxx/pqxx" @@ -18,10 +18,19 @@ public: const LSNPosition & start_lsn); void run(); + void createSubscription(); private: + void readString(const char * message, size_t & pos, size_t size, String & result); + Int64 readInt64(const char * message, size_t & pos); + Int32 readInt32(const char * message, size_t & pos); + Int16 readInt16(const char * message, size_t & pos); + Int8 readInt8(const char * message, size_t & pos); + void readTupleData(const char * message, size_t & pos, size_t size); + void startReplication( const std::string & slot_name, const std::string start_lsn, const int64_t timeline, const std::string & plugin_args); + void decodeReplicationMessage(const char * replication_message, size_t size); Poco::Logger * log; const std::string replication_slot_name; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 7fbcf1d9ff2..befaf17db11 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -35,11 +35,11 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( /// Used commands require a specific transaction isolation mode. replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); - /// Non temporary replication slot should be the same at restart. + /// Non temporary replication slot. Should be the same at restart. if (replication_slot.empty()) replication_slot = fmt::format("{}_{}_ch_replication_slot", database_name, table_name); - /// Temporary replication slot is used to determine a start lsn position and to acquire a snapshot for initial table synchronization. + /// Temporary replication slot is used to acquire a snapshot for initial table synchronization and to determine starting lsn position. temp_replication_slot = replication_slot + "_temp"; } @@ -51,6 +51,8 @@ void PostgreSQLReplicationHandler::startup() { publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); + /// Publication defines what tables are included into replication stream. Should be deleted only if MaterializePostgreSQL + /// table is dropped. if (!isPublicationExist()) createPublication(); } @@ -70,6 +72,7 @@ bool PostgreSQLReplicationHandler::isPublicationExist() { std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); pqxx::result result{tx->exec(query_str)}; + assert(!result.empty()); bool publication_exists = (result[0][0].as() == "t"); if (publication_exists) @@ -81,11 +84,7 @@ bool PostgreSQLReplicationHandler::isPublicationExist() void PostgreSQLReplicationHandler::createPublication() { - /* * It is also important that change replica identity for this table to be able to receive old values of updated rows: - * ALTER TABLE pgbench_accounts REPLICA IDENTITY FULL; - * * TRUNCATE and DDL are not included in PUBLICATION. - * * 'ONLY' means just a table, without descendants. - */ + /// 'ONLY' means just a table, without descendants. std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, table_name); try { @@ -96,6 +95,10 @@ void PostgreSQLReplicationHandler::createPublication() { throw Exception(fmt::format("PostgreSQL table {}.{} does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); } + + /// TODO: check replica identity + /// Requires changed replica identity for included table to be able to receive old values of updated rows. + /// (ALTER TABLE table_name REPLICA IDENTITY FULL) } @@ -103,7 +106,7 @@ void PostgreSQLReplicationHandler::startReplication() { auto ntx = std::make_shared(*replication_connection->conn()); - /// But it should not actually exist. May exist if failed to drop it before. + /// Normally temporary replication slot should not exist. if (isReplicationSlotExist(ntx, temp_replication_slot)) dropReplicationSlot(ntx, temp_replication_slot, true); @@ -116,10 +119,9 @@ void PostgreSQLReplicationHandler::startReplication() /// Do not need this replication slot anymore (snapshot loaded and start lsn determined, will continue replication protocol /// with another slot, which should be the same at restart (and reused) to minimize memory usage) - /// Non temporary replication slot should be deleted with drop table only. - LOG_DEBUG(log, "Dropping temporaty replication slot"); dropReplicationSlot(ntx, temp_replication_slot, true); + /// Non temporary replication slot should be deleted with drop table only. if (!isReplicationSlotExist(ntx, replication_slot)) createReplicationSlot(ntx); @@ -203,6 +205,8 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, st work.exec(query_str); work.commit(); } + + LOG_TRACE(log, "Replication slot {} is dropped", slot_name); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 06379f2ad4a..3dc94b0c776 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include "PostgreSQLConnection.h" #include "pqxx/pqxx" From 60ebb86d9745b7c2b9026b764efdfa1e4208369e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Jan 2021 15:29:28 +0000 Subject: [PATCH 003/105] Add stream and buffer classes --- .../PostgreSQLReplicaBlockInputStream.cpp | 149 ++++++++++++++++++ .../PostgreSQLReplicaBlockInputStream.h | 47 ++++++ .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 9 ++ .../PostgreSQL/PostgreSQLReplicaConsumer.h | 6 + .../PostgreSQLReplicaConsumerBuffer.cpp | 38 +++++ .../PostgreSQLReplicaConsumerBuffer.h | 39 +++++ .../PostgreSQLReplicationHandler.cpp | 3 + .../PostgreSQL/PostgreSQLReplicationHandler.h | 3 + .../PostgreSQLReplicationSettings.cpp | 4 +- .../PostgreSQLReplicationSettings.h | 6 +- .../PostgreSQL/StorageMaterializePostgreSQL.h | 40 ----- ...reSQL.cpp => StoragePostgreSQLReplica.cpp} | 54 +++++-- .../PostgreSQL/StoragePostgreSQLReplica.h | 57 +++++++ src/Storages/PostgreSQL/buffer_fwd.h | 9 ++ src/Storages/registerStorages.cpp | 4 +- 15 files changed, 406 insertions(+), 62 deletions(-) create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h delete mode 100644 src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h rename src/Storages/PostgreSQL/{StorageMaterializePostgreSQL.cpp => StoragePostgreSQLReplica.cpp} (66%) create mode 100644 src/Storages/PostgreSQL/StoragePostgreSQLReplica.h create mode 100644 src/Storages/PostgreSQL/buffer_fwd.h diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp new file mode 100644 index 00000000000..04ee68eb3aa --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp @@ -0,0 +1,149 @@ +#include "PostgreSQLReplicaBlockInputStream.h" + +#include +#include +#include + + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ + +PostgreSQLReplicaBlockInputStream::PostgreSQLReplicaBlockInputStream( + StoragePostgreSQLReplica & storage_, + ConsumerBufferPtr buffer_, + const StorageMetadataPtr & metadata_snapshot_, + std::shared_ptr context_, + const Names & columns, + size_t max_block_size_) + : storage(storage_) + , buffer(buffer_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) + , column_names(columns) + , max_block_size(max_block_size_) + , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) + , sample_block(non_virtual_header) + , virtual_header(metadata_snapshot->getSampleBlockForColumns({}, storage.getVirtuals(), storage.getStorageID())) +{ + for (const auto & column : virtual_header) + sample_block.insert(column); +} + + +PostgreSQLReplicaBlockInputStream::~PostgreSQLReplicaBlockInputStream() +{ +} + + +void PostgreSQLReplicaBlockInputStream::readPrefixImpl() +{ +} + + +Block PostgreSQLReplicaBlockInputStream::readImpl() +{ + if (!buffer || finished) + return Block(); + + finished = true; + + MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto input_format = FormatFactory::instance().getInputFormat( + "Values", *buffer, non_virtual_header, *context, max_block_size); + + InputPort port(input_format->getPort().getHeader(), input_format.get()); + connect(input_format->getPort(), port); + port.setNeeded(); + + auto read_rabbitmq_message = [&] + { + size_t new_rows = 0; + + while (true) + { + auto status = input_format->prepare(); + + switch (status) + { + case IProcessor::Status::Ready: + input_format->work(); + break; + + case IProcessor::Status::Finished: + input_format->resetParser(); + return new_rows; + + case IProcessor::Status::PortFull: + { + auto chunk = port.pull(); + + auto chunk_rows = chunk.getNumRows(); + new_rows += chunk_rows; + + auto columns = chunk.detachColumns(); + + for (size_t i = 0, s = columns.size(); i < s; ++i) + { + result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + } + break; + } + case IProcessor::Status::NeedData: + case IProcessor::Status::Async: + case IProcessor::Status::ExpandPipeline: + throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); + } + } + }; + + size_t total_rows = 0; + + while (true) + { + if (buffer->eof()) + break; + + auto new_rows = read_rabbitmq_message(); + + if (new_rows) + { + //auto timestamp = buffer->getTimestamp(); + //for (size_t i = 0; i < new_rows; ++i) + //{ + // virtual_columns[0]->insert(timestamp); + //} + + total_rows = total_rows + new_rows; + } + + buffer->allowNext(); + + if (total_rows >= max_block_size || !checkTimeLimit()) + break; + } + + if (total_rows == 0) + return Block(); + + auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); + + return result_block; +} + + +void PostgreSQLReplicaBlockInputStream::readSuffixImpl() +{ +} + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h b/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h new file mode 100644 index 00000000000..995c640682a --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include "StoragePostgreSQLReplica.h" +#include "PostgreSQLReplicaConsumerBuffer.h" +#include "buffer_fwd.h" + + +namespace DB +{ + +class PostgreSQLReplicaBlockInputStream : public IBlockInputStream +{ + +public: + PostgreSQLReplicaBlockInputStream( + StoragePostgreSQLReplica & storage_, + ConsumerBufferPtr buffer_, + const StorageMetadataPtr & metadata_snapshot_, + std::shared_ptr context_, + const Names & columns, + size_t max_block_size_); + + ~PostgreSQLReplicaBlockInputStream() override; + + String getName() const override { return storage.getName(); } + Block getHeader() const override { return sample_block; } + + void readPrefixImpl() override; + Block readImpl() override; + void readSuffixImpl() override; + +private: + StoragePostgreSQLReplica & storage; + ConsumerBufferPtr buffer; + StorageMetadataPtr metadata_snapshot; + std::shared_ptr context; + Names column_names; + const size_t max_block_size; + + bool finished = false; + const Block non_virtual_header; + Block sample_block; + const Block virtual_header; +}; + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 5010f574555..1d432bfa5cb 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -17,12 +17,14 @@ namespace ErrorCodes } PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( + Context & context_, const std::string & table_name_, const std::string & conn_str, const std::string & replication_slot_name_, const std::string & publication_name_, const LSNPosition & start_lsn) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) + , context(context_) , replication_slot_name(replication_slot_name_) , publication_name(publication_name_) , table_name(table_name_) @@ -30,6 +32,9 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( , current_lsn(start_lsn) { replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); + + wal_reader_task = context.getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ WALReaderFunc(); }); + wal_reader_task->deactivate(); } @@ -221,6 +226,10 @@ void PostgreSQLReplicaConsumer::run() } +void PostgreSQLReplicaConsumer::WALReaderFunc() +{ +} + } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index e2833676412..d64c11d72c4 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -2,6 +2,7 @@ #include "PostgreSQLConnection.h" #include "PostgreSQLReplicationHandler.h" +#include #include "pqxx/pqxx" namespace DB @@ -11,6 +12,7 @@ class PostgreSQLReplicaConsumer { public: PostgreSQLReplicaConsumer( + Context & context_, const std::string & table_name_, const std::string & conn_str_, const std::string & replication_slot_name_, @@ -32,7 +34,10 @@ private: const std::string & slot_name, const std::string start_lsn, const int64_t timeline, const std::string & plugin_args); void decodeReplicationMessage(const char * replication_message, size_t size); + void WALReaderFunc(); + Poco::Logger * log; + Context & context; const std::string replication_slot_name; const std::string publication_name; @@ -40,6 +45,7 @@ private: PostgreSQLConnectionPtr connection, replication_connection; LSNPosition current_lsn; + BackgroundSchedulePool::TaskHolder wal_reader_task; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp new file mode 100644 index 00000000000..e8c4ba3d55a --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp @@ -0,0 +1,38 @@ +#include "PostgreSQLReplicaConsumerBuffer.h" + + +namespace DB +{ + +PostgreSQLReplicaConsumerBuffer::PostgreSQLReplicaConsumerBuffer( + uint64_t max_block_size_) + : ReadBuffer(nullptr, 0) + , rows_data(max_block_size_) +{ +} + + +PostgreSQLReplicaConsumerBuffer::~PostgreSQLReplicaConsumerBuffer() +{ + BufferBase::set(nullptr, 0, 0); +} + + +bool PostgreSQLReplicaConsumerBuffer::nextImpl() +{ + if (!allowed) + return false; + + if (rows_data.tryPop(current_row_data)) + { + auto * new_position = const_cast(current_row_data.data.data()); + BufferBase::set(new_position, current_row_data.data.size(), 0); + allowed = false; + + return true; + } + + return false; +} + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h new file mode 100644 index 00000000000..8c8de3a8b68 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include +#include +#include "buffer_fwd.h" + + +namespace DB +{ + +class PostgreSQLReplicaConsumerBuffer : public ReadBuffer +{ + +public: + PostgreSQLReplicaConsumerBuffer( + uint64_t max_block_size_); + + ~PostgreSQLReplicaConsumerBuffer() override; + + void allowNext() { allowed = true; } + +private: + bool nextImpl() override; + + struct RowData + { + String data; + RowData() : data("") {} + }; + + RowData current_row_data; + ConcurrentBoundedQueue rows_data; + bool allowed = true; +}; + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index befaf17db11..18b6aca5de1 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -15,12 +15,14 @@ namespace ErrorCodes } PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( + Context & context_, const std::string & database_name_, const std::string & table_name_, const std::string & conn_str, const std::string & replication_slot_, const std::string & publication_name_) : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) + , context(context_) , database_name(database_name_) , table_name(table_name_) , replication_slot(replication_slot_) @@ -126,6 +128,7 @@ void PostgreSQLReplicationHandler::startReplication() createReplicationSlot(ntx); PostgreSQLReplicaConsumer consumer( + context, table_name, connection->conn_str(), replication_slot, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 3dc94b0c776..55ea8dbae41 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -2,6 +2,7 @@ #include #include "PostgreSQLConnection.h" +#include #include "pqxx/pqxx" @@ -33,6 +34,7 @@ class PostgreSQLReplicationHandler public: friend class PGReplicaLSN; PostgreSQLReplicationHandler( + Context & context_, const std::string & database_name_, const std::string & table_name_, const std::string & conn_str_, @@ -57,6 +59,7 @@ private: void loadFromSnapshot(std::string & snapshot_name); Poco::Logger * log; + Context & context; const std::string database_name, table_name; std::string replication_slot, publication_name; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp index 98173f7ca07..fa5ebb0edf3 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp @@ -13,9 +13,9 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(PostgreSQLReplicationSettingsTraits, LIST_OF_POSTGRESQL_REPLICATION_SETTINGS) -void MaterializePostgreSQLSettings::loadFromQuery(ASTStorage & storage_def) +void PostgreSQLReplicationSettings::loadFromQuery(ASTStorage & storage_def) { if (storage_def.settings) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h index c85b3572356..1c3ca6ff73d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h @@ -7,13 +7,13 @@ namespace DB class ASTStorage; -#define LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS(M) \ +#define LIST_OF_POSTGRESQL_REPLICATION_SETTINGS(M) \ M(String, postgresql_replication_slot_name, "", "PostgreSQL replication slot name.", 0) \ M(String, postgresql_publication_name, "", "PostgreSQL publication name.", 0) \ -DECLARE_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) +DECLARE_SETTINGS_TRAITS(PostgreSQLReplicationSettingsTraits, LIST_OF_POSTGRESQL_REPLICATION_SETTINGS) -struct MaterializePostgreSQLSettings : public BaseSettings +struct PostgreSQLReplicationSettings : public BaseSettings { void loadFromQuery(ASTStorage & storage_def); }; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h deleted file mode 100644 index ef0eb4e75cd..00000000000 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ /dev/null @@ -1,40 +0,0 @@ -#pragma once - -#include "config_core.h" - -#include -#include -#include -#include "PostgreSQLReplicationHandler.h" -#include "pqxx/pqxx" - -namespace DB -{ - -class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage -{ - friend struct ext::shared_ptr_helper; - -public: - StorageMaterializePostgreSQL( - const StorageID & table_id_, - const String & remote_table_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const Context & context_, - std::shared_ptr replication_handler_); - - String getName() const override { return "MaterializePostgreSQL"; } - - void startup() override; - void shutdown() override; - -private: - String remote_table_name; - Context global_context; - - std::shared_ptr replication_handler; -}; - -} - diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp similarity index 66% rename from src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp rename to src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 385eb8a8706..657f8e5aa47 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -1,9 +1,10 @@ -#include "StorageMaterializePostgreSQL.h" +#include "StoragePostgreSQLReplica.h" #include #include + #include -#include + #include #include #include @@ -12,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +21,7 @@ #include #include "PostgreSQLReplicationSettings.h" #include +#include "PostgreSQLReplicaBlockInputStream.h" namespace DB @@ -29,51 +32,66 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( +StoragePostgreSQLReplica::StoragePostgreSQLReplica( const StorageID & table_id_, const String & remote_table_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - std::shared_ptr replication_handler_) + const PostgreSQLReplicationHandler & replication_handler_, + std::unique_ptr replication_settings_) : IStorage(table_id_) , remote_table_name(remote_table_name_) , global_context(context_) - , replication_handler(std::move(replication_handler_)) + , replication_settings(std::move(replication_settings_)) + , replication_handler(std::make_unique(replication_handler_)) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - } -void StorageMaterializePostgreSQL::startup() +void StoragePostgreSQLReplica::startup() { replication_handler->startup(); } -void StorageMaterializePostgreSQL::shutdown() +void StoragePostgreSQLReplica::shutdown() { //replication_handler->dropReplicationSlot(); } -void registerStorageMaterializePostgreSQL(StorageFactory & factory) +Pipe StoragePostgreSQLReplica::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & /* query_info */, + const Context & /* context */, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) +{ + auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + return Pipe(); +} + + +void registerStoragePostgreSQLReplica(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; bool has_settings = args.storage_def->settings; - auto postgresql_replication_settings = std::make_unique(); + auto postgresql_replication_settings = std::make_unique(); if (has_settings) postgresql_replication_settings->loadFromQuery(*args.storage_def); if (engine_args.size() != 5) - throw Exception("Storage MaterializePostgreSQL requires 5 parameters: " + throw Exception("Storage PostgreSQLReplica requires 5 parameters: " "PostgreSQL('host:port', 'database', 'table', 'username', 'password'", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -95,19 +113,25 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) auto replication_slot_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_replication_slot_name.value); auto publication_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_publication_name.value); - PostgreSQLReplicationHandler replication_handler(remote_database, remote_table, connection_str, replication_slot_name, publication_name); + PostgreSQLReplicationHandler replication_handler(global_context, remote_database, remote_table, connection_str, replication_slot_name, publication_name); - return StorageMaterializePostgreSQL::create( + return StoragePostgreSQLReplica::create( args.table_id, remote_table, args.columns, args.constraints, global_context, - std::make_shared(replication_handler)); + replication_handler, std::move(postgresql_replication_settings)); }; factory.registerStorage( - "MaterializePostgreSQL", + "PostgreSQLReplica", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, .source_access_type = AccessType::POSTGRES, }); } +NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const +{ + return NamesAndTypesList{ + }; +} + } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h new file mode 100644 index 00000000000..860ab8f6d6c --- /dev/null +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -0,0 +1,57 @@ +#pragma once + +#include "config_core.h" + +#include +#include +#include +#include "PostgreSQLReplicationHandler.h" +#include "PostgreSQLReplicationSettings.h" +#include "buffer_fwd.h" +#include "pqxx/pqxx" + +namespace DB +{ + +class StoragePostgreSQLReplica final : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; + +public: + String getName() const override { return "PostgreSQLReplica"; } + + void startup() override; + void shutdown() override; + + NamesAndTypesList getVirtuals() const override; + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + +protected: + StoragePostgreSQLReplica( + const StorageID & table_id_, + const String & remote_table_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const Context & context_, + const PostgreSQLReplicationHandler & replication_handler_, + std::unique_ptr replication_settings_); + +private: + String remote_table_name; + Context global_context; + + std::unique_ptr replication_settings; + std::unique_ptr replication_handler; +}; + +} + diff --git a/src/Storages/PostgreSQL/buffer_fwd.h b/src/Storages/PostgreSQL/buffer_fwd.h new file mode 100644 index 00000000000..40ffd64aad3 --- /dev/null +++ b/src/Storages/PostgreSQL/buffer_fwd.h @@ -0,0 +1,9 @@ +#pragma once + +namespace DB +{ + +class PostgreSQLReplicaConsumerBuffer; +using ConsumerBufferPtr = std::shared_ptr; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index bd32de1c315..0f5a3acaa86 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -60,7 +60,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory); #if USE_LIBPQXX void registerStoragePostgreSQL(StorageFactory & factory); -void registerStorageMaterializePostgreSQL(StorageFactory & factory); +void registerStoragePostgreSQLReplica(StorageFactory & factory); #endif void registerStorages() @@ -118,7 +118,7 @@ void registerStorages() #if USE_LIBPQXX registerStoragePostgreSQL(factory); - registerStorageMaterializePostgreSQL(factory); + registerStoragePostgreSQLReplica(factory); #endif } From b842ce114bc1c163cd1a7a67d1de8687ce136015 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Jan 2021 21:46:19 +0000 Subject: [PATCH 004/105] Initial sync into ReplacingMergeTree table, select via nested table --- .../compose/docker_compose_postgres.yml | 1 + src/Core/Settings.h | 1 + .../PostgreSQLBlockInputStream.cpp | 5 +- src/DataStreams/PostgreSQLBlockInputStream.h | 4 +- .../PostgreSQLDictionarySource.cpp | 15 +- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 113 ++++++--- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 45 +++- .../PostgreSQLReplicationHandler.cpp | 113 +++++---- .../PostgreSQL/PostgreSQLReplicationHandler.h | 28 +-- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 226 ++++++++++++++---- .../PostgreSQL/StoragePostgreSQLReplica.h | 29 ++- src/Storages/StoragePostgreSQL.cpp | 3 +- .../__init__.py | 0 .../configs/log_conf.xml | 11 + .../test_storage_postgresql_replica/test.py | 87 +++++++ .../test_postgresql_replica.reference | 50 ++++ 16 files changed, 575 insertions(+), 156 deletions(-) create mode 100644 tests/integration/test_storage_postgresql_replica/__init__.py create mode 100644 tests/integration/test_storage_postgresql_replica/configs/log_conf.xml create mode 100644 tests/integration/test_storage_postgresql_replica/test.py create mode 100644 tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index fff4fb1fa42..7b3bee8de08 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -7,6 +7,7 @@ services: POSTGRES_PASSWORD: mysecretpassword ports: - 5432:5432 + command: [ "postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=5"] networks: default: aliases: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9bb9ad30f15..8c830f2dac1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -60,6 +60,7 @@ class IColumn; M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \ M(Milliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.", 0) \ M(Milliseconds, rabbitmq_max_wait_ms, 5000, "The wait time for reading from RabbitMQ before retry.", 0) \ + M(UInt64, postgresql_replica_max_rows_to_insert, 65536, "Maximum number of rows in PostgreSQL batch insertion in PostgreSQLReplica storage engine", 0) \ M(UInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.", 0) \ M(UInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \ M(UInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \ diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index da6a83fb930..4646a8a9d32 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -28,13 +28,13 @@ namespace ErrorCodes } PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( - ConnectionPtr connection_, + std::unique_ptr tx_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_) : query_str(query_str_) , max_block_size(max_block_size_) - , connection(connection_) + , tx(std::move(tx_)) { description.init(sample_block); for (const auto idx : ext::range(0, description.sample_block.columns())) @@ -48,7 +48,6 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( void PostgreSQLBlockInputStream::readPrefix() { - tx = std::make_unique(*connection); stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index b88c81cca0a..1e52b48c7cf 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -20,7 +20,7 @@ class PostgreSQLBlockInputStream : public IBlockInputStream { public: PostgreSQLBlockInputStream( - ConnectionPtr connection_, + std::unique_ptr tx_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_); @@ -48,7 +48,7 @@ private: ExternalResultDescription description; ConnectionPtr connection; - std::unique_ptr tx; + std::unique_ptr tx; std::unique_ptr stream; struct ArrayInfo diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index aa852404750..8ede0bc8813 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -68,8 +68,9 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); + auto tx = std::make_unique(*connection->conn()); return std::make_shared( - connection->conn(), load_all_query, sample_block, max_block_size); + std::move(tx), load_all_query, sample_block, max_block_size); } @@ -77,20 +78,23 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); - return std::make_shared(connection->conn(), load_update_query, sample_block, max_block_size); + auto tx = std::make_unique(*connection->conn()); + return std::make_shared(std::move(tx), load_update_query, sample_block, max_block_size); } BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); - return std::make_shared(connection->conn(), query, sample_block, max_block_size); + auto tx = std::make_unique(*connection->conn()); + return std::make_shared(std::move(tx), query, sample_block, max_block_size); } BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); - return std::make_shared(connection->conn(), query, sample_block, max_block_size); + auto tx = std::make_unique(*connection->conn()); + return std::make_shared(std::move(tx), query, sample_block, max_block_size); } @@ -112,7 +116,8 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - PostgreSQLBlockInputStream block_input_stream(connection->conn(), request, invalidate_sample_block, 1); + auto tx = std::make_unique(*connection->conn()); + PostgreSQLBlockInputStream block_input_stream(std::move(tx), request, invalidate_sample_block, 1); return readInvalidateQuery(block_input_stream); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 1d432bfa5cb..076863eb8dd 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -16,8 +17,12 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static const auto wal_reader_reschedule_ms = 500; +static const auto max_thread_work_duration_ms = 60000; +static const auto max_empty_slot_reads = 20; + PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( - Context & context_, + std::shared_ptr context_, const std::string & table_name_, const std::string & conn_str, const std::string & replication_slot_name_, @@ -33,11 +38,56 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( { replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); - wal_reader_task = context.getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ WALReaderFunc(); }); + wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ WALReaderFunc(); }); wal_reader_task->deactivate(); } +void PostgreSQLReplicaConsumer::startSynchronization() +{ + //wal_reader_task->activateAndSchedule(); +} + + +void PostgreSQLReplicaConsumer::stopSynchronization() +{ + stop_synchronization.store(true); + if (wal_reader_task) + wal_reader_task->deactivate(); +} + + +void PostgreSQLReplicaConsumer::WALReaderFunc() +{ + size_t count_empty_slot_reads = 0; + auto start_time = std::chrono::steady_clock::now(); + + LOG_TRACE(log, "Starting synchronization thread"); + + while (!stop_synchronization) + { + if (!readFromReplicationSlot() && ++count_empty_slot_reads == max_empty_slot_reads) + { + LOG_TRACE(log, "Reschedule synchronization. Replication slot is empty."); + break; + } + else + count_empty_slot_reads = 0; + + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (duration.count() > max_thread_work_duration_ms) + { + LOG_TRACE(log, "Reschedule synchronization. Thread work duration limit exceeded."); + break; + } + } + + if (!stop_synchronization) + wal_reader_task->scheduleAfter(wal_reader_reschedule_ms); +} + + void PostgreSQLReplicaConsumer::readString(const char * message, size_t & pos, size_t size, String & result) { assert(size > pos + 2); @@ -195,39 +245,48 @@ void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replicatio } -void PostgreSQLReplicaConsumer::run() +/// Read binary changes from replication slot via copy command. +bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { - auto tx = std::make_unique(*replication_connection->conn()); - /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. - std::string query_str = fmt::format( - "select data FROM pg_logical_slot_peek_binary_changes(" - "'{}', NULL, NULL, 'publication_names', '{}', 'proto_version', '1')", - replication_slot_name, publication_name); - pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); - - while (true) + bool slot_empty = true; + try { - const std::vector * row{stream.read_row()}; + auto tx = std::make_unique(*replication_connection->conn()); + /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. + std::string query_str = fmt::format( + "select data FROM pg_logical_slot_peek_binary_changes(" + "'{}', NULL, NULL, 'publication_names', '{}', 'proto_version', '1')", + replication_slot_name, publication_name); + pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); - if (!row) + while (true) { - LOG_TRACE(log, "STREAM REPLICATION END"); - stream.complete(); - tx->commit(); - break; - } + const std::vector * row{stream.read_row()}; - for (const auto idx : ext::range(0, row->size())) - { - LOG_TRACE(log, "Replication message: {}", (*row)[idx]); - decodeReplicationMessage((*row)[idx].c_str(), (*row)[idx].size()); + if (!row) + { + LOG_TRACE(log, "STREAM REPLICATION END"); + stream.complete(); + tx->commit(); + break; + } + + slot_empty = false; + + for (const auto idx : ext::range(0, row->size())) + { + LOG_TRACE(log, "Replication message: {}", (*row)[idx]); + decodeReplicationMessage((*row)[idx].c_str(), (*row)[idx].size()); + } } } -} + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + return false; + } - -void PostgreSQLReplicaConsumer::WALReaderFunc() -{ + return !slot_empty; } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index d64c11d72c4..bb4b4c5033b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -1,43 +1,63 @@ #pragma once #include "PostgreSQLConnection.h" -#include "PostgreSQLReplicationHandler.h" #include +#include #include "pqxx/pqxx" namespace DB { +struct LSNPosition +{ + std::string lsn; + + uint64_t getValue() + { + uint64_t upper_half, lower_half, result; + std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); + result = (upper_half << 32) + lower_half; + //LOG_DEBUG(&Poco::Logger::get("LSNParsing"), + // "Created replication slot. upper half: {}, lower_half: {}, start lsn: {}", + // upper_half, lower_half, result); + return result; + } +}; + + class PostgreSQLReplicaConsumer { public: PostgreSQLReplicaConsumer( - Context & context_, + std::shared_ptr context_, const std::string & table_name_, const std::string & conn_str_, const std::string & replication_slot_name_, const std::string & publication_name_, const LSNPosition & start_lsn); - void run(); - void createSubscription(); + /// Start reading WAL from current_lsn position. Initial data sync from created snapshot already done. + void startSynchronization(); + void stopSynchronization(); private: + /// Executed by wal_reader_task. A separate thread reads wal and advances lsn when rows were written via copyData. + void WALReaderFunc(); + + /// Start changes stream from WAL via copy command (up to max_block_size changes). + bool readFromReplicationSlot(); + void decodeReplicationMessage(const char * replication_message, size_t size); + + /// Methods to parse replication message data. + void readTupleData(const char * message, size_t & pos, size_t size); void readString(const char * message, size_t & pos, size_t size, String & result); Int64 readInt64(const char * message, size_t & pos); Int32 readInt32(const char * message, size_t & pos); Int16 readInt16(const char * message, size_t & pos); Int8 readInt8(const char * message, size_t & pos); - void readTupleData(const char * message, size_t & pos, size_t size); - - void startReplication( - const std::string & slot_name, const std::string start_lsn, const int64_t timeline, const std::string & plugin_args); - void decodeReplicationMessage(const char * replication_message, size_t size); - - void WALReaderFunc(); Poco::Logger * log; - Context & context; + std::shared_ptr context; const std::string replication_slot_name; const std::string publication_name; @@ -46,6 +66,7 @@ private: LSNPosition current_lsn; BackgroundSchedulePool::TaskHolder wal_reader_task; + std::atomic stop_synchronization = false; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 18b6aca5de1..f6571ce5a2e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,8 +1,14 @@ #include "PostgreSQLReplicationHandler.h" #include "PostgreSQLReplicaConsumer.h" +#include #include #include +#include +#include +#include +#include +#include namespace DB { @@ -15,14 +21,12 @@ namespace ErrorCodes } PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( - Context & context_, const std::string & database_name_, const std::string & table_name_, const std::string & conn_str, const std::string & replication_slot_, const std::string & publication_name_) : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) - , context(context_) , database_name(database_name_) , table_name(table_name_) , replication_slot(replication_slot_) @@ -46,8 +50,10 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } -void PostgreSQLReplicationHandler::startup() +void PostgreSQLReplicationHandler::startup(StoragePtr storage, std::shared_ptr context_) { + helper_table = storage; + context = context_; tx = std::make_shared(*connection->conn()); if (publication_name.empty()) { @@ -70,6 +76,13 @@ void PostgreSQLReplicationHandler::startup() } +void PostgreSQLReplicationHandler::shutdown() +{ + if (consumer) + consumer->stopSynchronization(); +} + + bool PostgreSQLReplicationHandler::isPublicationExist() { std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); @@ -115,19 +128,20 @@ void PostgreSQLReplicationHandler::startReplication() std::string snapshot_name; LSNPosition start_lsn; - createTempReplicationSlot(ntx, start_lsn, snapshot_name); - - loadFromSnapshot(snapshot_name); - - /// Do not need this replication slot anymore (snapshot loaded and start lsn determined, will continue replication protocol - /// with another slot, which should be the same at restart (and reused) to minimize memory usage) - dropReplicationSlot(ntx, temp_replication_slot, true); - - /// Non temporary replication slot should be deleted with drop table only. + /// Non temporary replication slot should be deleted with drop table only and created only once, reused after detach. if (!isReplicationSlotExist(ntx, replication_slot)) + { + /// Temporary replication slot + createTempReplicationSlot(ntx, start_lsn, snapshot_name); + /// Initial table synchronization from created snapshot + loadFromSnapshot(snapshot_name); + /// Do not need this replication slot anymore (snapshot loaded and start lsn determined + dropReplicationSlot(ntx, temp_replication_slot, true); + /// Non-temporary replication slot createReplicationSlot(ntx); + } - PostgreSQLReplicaConsumer consumer( + consumer = std::make_shared( context, table_name, connection->conn_str(), @@ -138,7 +152,50 @@ void PostgreSQLReplicationHandler::startReplication() LOG_DEBUG(log, "Commiting replication transaction"); ntx->commit(); - consumer.run(); + consumer->startSynchronization(); +} + + +void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) +{ + LOG_DEBUG(log, "Creating transaction snapshot"); + + try + { + auto stx = std::make_unique(*connection->conn()); + + /// Specific isolation level is required to read from snapshot. + stx->set_variable("transaction_isolation", "'repeatable read'"); + + std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); + stx->exec(query_str); + + /// Load from snapshot, which will show table state before creation of replication slot. + query_str = fmt::format("SELECT * FROM {}", table_name); + + Context insert_context(*context); + insert_context.makeQueryContext(); + + auto insert = std::make_shared(); + insert->table_id = helper_table->getStorageID(); + + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); + + const StorageInMemoryMetadata & storage_metadata = helper_table->getInMemoryMetadata(); + auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); + + PostgreSQLBlockInputStream input(std::move(stx), query_str, sample_block, DEFAULT_BLOCK_SIZE); + + copyData(input, *block_io.out); + } + catch (Exception & e) + { + e.addMessage("while initial data sync for table {}.{}", database_name, table_name); + throw; + } + + LOG_DEBUG(log, "Done loading from snapshot"); } @@ -223,32 +280,4 @@ void PostgreSQLReplicationHandler::checkAndDropReplicationSlot() } -void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) -{ - auto stx = std::make_unique(*connection->conn()); - /// Required to execute the following command. - stx->set_variable("transaction_isolation", "'repeatable read'"); - - std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); - stx->exec(query_str); - - LOG_DEBUG(log, "Created transaction snapshot"); - query_str = fmt::format("SELECT * FROM {}", table_name); - pqxx::result result{stx->exec(query_str)}; - if (!result.empty()) - { - pqxx::row row{result[0]}; - for (auto res : row) - { - if (std::size(res)) - LOG_TRACE(log, "GOT {}", res.as()); - else - LOG_TRACE(log, "GOT NULL"); - } - } - LOG_DEBUG(log, "Done loading from snapshot"); - stx->commit(); -} - - } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 55ea8dbae41..af4465b863a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -2,6 +2,7 @@ #include #include "PostgreSQLConnection.h" +#include "PostgreSQLReplicaConsumer.h" #include #include "pqxx/pqxx" @@ -12,36 +13,19 @@ namespace DB { -struct LSNPosition -{ - std::string lsn; - - uint64_t getValue() - { - uint64_t upper_half, lower_half, result; - std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); - result = (upper_half << 32) + lower_half; - LOG_DEBUG(&Poco::Logger::get("LSNParsing"), - "Created replication slot. upper half: {}, lower_half: {}, start lsn: {}", - upper_half, lower_half, result); - return result; - } -}; - - class PostgreSQLReplicationHandler { public: friend class PGReplicaLSN; PostgreSQLReplicationHandler( - Context & context_, const std::string & database_name_, const std::string & table_name_, const std::string & conn_str_, const std::string & replication_slot_name_, const std::string & publication_name_); - void startup(); + void startup(StoragePtr storage_, std::shared_ptr context_); + void shutdown(); void checkAndDropReplicationSlot(); private: @@ -57,9 +41,10 @@ private: void startReplication(); void loadFromSnapshot(std::string & snapshot_name); + Context createQueryContext(); + void getTableOutput(const Context & query_context); Poco::Logger * log; - Context & context; const std::string database_name, table_name; std::string replication_slot, publication_name; @@ -69,6 +54,9 @@ private: PostgreSQLConnectionPtr replication_connection; std::shared_ptr tx; + std::shared_ptr consumer; + std::shared_ptr context; + StoragePtr helper_table; //LSNPosition start_lsn, final_lsn; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 657f8e5aa47..9ae90e24429 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -1,28 +1,36 @@ #include "StoragePostgreSQLReplica.h" -#include -#include - #include +#include +#include -#include -#include -#include -#include -#include +#include #include #include -#include -#include -#include -#include + +#include +#include + #include #include -#include -#include "PostgreSQLReplicationSettings.h" -#include -#include "PostgreSQLReplicaBlockInputStream.h" +#include +#include + +#include +#include +#include +#include +#include + +#include + +#include "PostgreSQLReplicationSettings.h" +#include "PostgreSQLReplicaBlockInputStream.h" +#include + +#include +#include namespace DB { @@ -32,50 +40,177 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +static auto nested_storage_suffix = "_ReplacingMergeTree"; + StoragePostgreSQLReplica::StoragePostgreSQLReplica( const StorageID & table_id_, const String & remote_table_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, + const String & relative_data_path_, + const StorageInMemoryMetadata & storage_metadata, const Context & context_, const PostgreSQLReplicationHandler & replication_handler_, std::unique_ptr replication_settings_) : IStorage(table_id_) , remote_table_name(remote_table_name_) - , global_context(context_) + , relative_data_path(relative_data_path_) + , global_context(std::make_shared(context_)) , replication_settings(std::move(replication_settings_)) , replication_handler(std::make_unique(replication_handler_)) { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); + relative_data_path.resize(relative_data_path.size() - 1); + relative_data_path += nested_storage_suffix; +} + + +std::shared_ptr StoragePostgreSQLReplica::getMaterializedColumnsDeclaration( + const String name, const String type, UInt64 default_value) +{ + auto column_declaration = std::make_shared(); + + column_declaration->name = name; + column_declaration->type = makeASTFunction(type); + + column_declaration->default_specifier = "MATERIALIZED"; + column_declaration->default_expression = std::make_shared(default_value); + + column_declaration->children.emplace_back(column_declaration->type); + column_declaration->children.emplace_back(column_declaration->default_expression); + + return column_declaration; +} + + +ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_type) +{ + WhichDataType which(data_type); + + if (which.isNullable()) + return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + + if (which.isArray()) + return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + + return std::make_shared(data_type->getName()); +} + + +std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage() +{ + auto columns_declare_list = std::make_shared(); + + auto columns_expression_list = std::make_shared(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) + { + const auto & column_declaration = std::make_shared(); + column_declaration->name = column_type_and_name.name; + column_declaration->type = getColumnDeclaration(column_type_and_name.type); + columns_expression_list->children.emplace_back(column_declaration); + } + columns_declare_list->set(columns_declare_list->columns, columns_expression_list); + + columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", UInt64(1))); + columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", UInt64(1))); + + return columns_declare_list; +} + + +ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() +{ + auto create_table_query = std::make_shared(); + + auto table_id = getStorageID(); + create_table_query->table = table_id.table_name + nested_storage_suffix; + create_table_query->database = table_id.database_name; + create_table_query->if_not_exists = true; + + create_table_query->set(create_table_query->columns_list, getColumnsListFromStorage()); + + auto storage = std::make_shared(); + storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared("_version"))); + + auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST(); + if (primary_key_ast) + storage->set(storage->order_by, primary_key_ast); + /// else + + //storage->set(storage->partition_by, ?); + + create_table_query->set(create_table_query->storage, storage); + + return create_table_query; } void StoragePostgreSQLReplica::startup() { - replication_handler->startup(); + Context context_copy(*global_context); + const auto ast_create = getCreateHelperTableQuery(); + + Poco::File path(relative_data_path); + if (!path.exists()) + { + LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), + "Creating helper table {}", getStorageID().table_name + nested_storage_suffix); + InterpreterCreateQuery interpreter(ast_create, context_copy); + interpreter.execute(); + } + + nested_storage = createTableFromAST(ast_create->as(), getStorageID().database_name, relative_data_path, context_copy, false).second; + nested_storage->startup(); + + replication_handler->startup(nested_storage, global_context); +} + + +void StoragePostgreSQLReplica::drop() +{ + /// TODO: Under lock? Make sure synchronization stopped. + replication_handler->checkAndDropReplicationSlot(); + + nested_storage->drop(); + + relative_data_path.resize(relative_data_path.size() - 1); + Poco::File path(relative_data_path); + if (path.exists()) + path.remove(true); } void StoragePostgreSQLReplica::shutdown() { - //replication_handler->dropReplicationSlot(); + replication_handler->shutdown(); } Pipe StoragePostgreSQLReplica::read( const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - SelectQueryInfo & /* query_info */, - const Context & /* context */, - QueryProcessingStage::Enum /* processed_stage */, - size_t /* max_block_size */, - unsigned /* num_streams */) + const StorageMetadataPtr & /* metadata_snapshot */, + SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) { - auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); - return Pipe(); + StoragePtr storage = DatabaseCatalog::instance().getTable(nested_storage->getStorageID(), *global_context); + auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + + const StorageMetadataPtr & nested_metadata = storage->getInMemoryMetadataPtr(); + Pipe pipe = storage->read( + column_names, + nested_metadata, query_info, context, + processed_stage, max_block_size, num_streams); + + pipe.addTableLock(lock); + return pipe; +} + + +NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const +{ + return NamesAndTypesList{}; } @@ -98,6 +233,21 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context); + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->order_by && args.storage_def->primary_key) + args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); + + if (!args.storage_def->order_by) + throw Exception("Storage PostgreSQLReplica needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS); + + if (args.storage_def->primary_key) + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); + else + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context); + auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 5432); const String & remote_table = engine_args[2]->as().value.safeGet(); const String & remote_database = engine_args[1]->as().value.safeGet(); @@ -113,25 +263,19 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) auto replication_slot_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_replication_slot_name.value); auto publication_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_publication_name.value); - PostgreSQLReplicationHandler replication_handler(global_context, remote_database, remote_table, connection_str, replication_slot_name, publication_name); + PostgreSQLReplicationHandler replication_handler(remote_database, remote_table, connection_str, replication_slot_name, publication_name); return StoragePostgreSQLReplica::create( - args.table_id, remote_table, args.columns, args.constraints, global_context, + args.table_id, remote_table, args.relative_data_path, metadata, global_context, replication_handler, std::move(postgresql_replication_settings)); }; factory.registerStorage( "PostgreSQLReplica", creator_fn, - StorageFactory::StorageFeatures{ .supports_settings = true, .source_access_type = AccessType::POSTGRES, + StorageFactory::StorageFeatures{ .supports_settings = true, .supports_sort_order = true, .source_access_type = AccessType::POSTGRES, }); } -NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const -{ - return NamesAndTypesList{ - }; -} - } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 860ab8f6d6c..18b0504d32c 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -2,6 +2,18 @@ #include "config_core.h" +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + #include #include #include @@ -21,6 +33,7 @@ public: String getName() const override { return "PostgreSQLReplica"; } void startup() override; + void drop() override; void shutdown() override; NamesAndTypesList getVirtuals() const override; @@ -39,18 +52,28 @@ protected: StoragePostgreSQLReplica( const StorageID & table_id_, const String & remote_table_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, + const String & relative_data_path_, + const StorageInMemoryMetadata & storage_metadata, const Context & context_, const PostgreSQLReplicationHandler & replication_handler_, std::unique_ptr replication_settings_); private: + std::shared_ptr getMaterializedColumnsDeclaration( + const String name, const String type, UInt64 default_value); + std::shared_ptr getColumnsListFromStorage(); + ASTPtr getColumnDeclaration(const DataTypePtr & data_type); + ASTPtr getCreateHelperTableQuery(); + String remote_table_name; - Context global_context; + String relative_data_path; + std::shared_ptr global_context; std::unique_ptr replication_settings; std::unique_ptr replication_handler; + + /// ReplacingMergeTree table + StoragePtr nested_storage; }; } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 78ec8c34e41..08affa518e7 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -83,8 +83,9 @@ Pipe StoragePostgreSQL::read( sample_block.insert({ column_data.type, column_data.name }); } + auto tx = std::make_unique(*connection->conn()); return Pipe(std::make_shared( - std::make_shared(connection->conn(), query, sample_block, max_block_size_))); + std::make_shared(std::move(tx), query, sample_block, max_block_size_))); } diff --git a/tests/integration/test_storage_postgresql_replica/__init__.py b/tests/integration/test_storage_postgresql_replica/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml b/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml new file mode 100644 index 00000000000..f9d15e572aa --- /dev/null +++ b/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py new file mode 100644 index 00000000000..3d98486fb4b --- /dev/null +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -0,0 +1,87 @@ +import pytest +import time +import psycopg2 +import os.path as p + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True) + +postgres_table_template = """ + CREATE TABLE IF NOT EXISTS {} ( + key Integer NOT NULL, value Integer, PRIMARY KEY (key)) + """ + +def get_postgres_conn(database=False): + if database == True: + conn_string = "host='localhost' dbname='postgres_database' user='postgres' password='mysecretpassword'" + else: + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + +def create_postgres_db(cursor, name): + cursor.execute("CREATE DATABASE {}".format(name)) + +def create_postgres_table(cursor, table_name): + cursor.execute(postgres_table_template.format(table_name)) + +def postgresql_replica_check_result(result, check=False, ref_file='test_postgresql_replica.reference'): + fpath = p.join(p.dirname(__file__), ref_file) + with open(fpath) as reference: + if check: + assert TSV(result) == TSV(reference) + else: + return TSV(result) == TSV(reference) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + conn = get_postgres_conn() + cursor = conn.cursor() + create_postgres_db(cursor, 'postgres_database') + instance.query(''' + CREATE DATABASE postgres_database + ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')''') + + instance.query('CREATE DATABASE test') + yield cluster + + finally: + cluster.shutdown() + +@pytest.fixture(autouse=True) +def rabbitmq_setup_teardown(): + yield # run test + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') + + +def test_initial_load_from_snapshot(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + result = instance.query('SELECT * FROM test.postgresql_replica;') + postgresql_replica_check_result(result, True) + + +if __name__ == '__main__': + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() diff --git a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference b/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference new file mode 100644 index 00000000000..959bb2aad74 --- /dev/null +++ b/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference @@ -0,0 +1,50 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 +32 32 +33 33 +34 34 +35 35 +36 36 +37 37 +38 38 +39 39 +40 40 +41 41 +42 42 +43 43 +44 44 +45 45 +46 46 +47 47 +48 48 +49 49 From 179a558a048162d9ca40b091da3e4633f4269a65 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 30 Jan 2021 12:05:14 +0000 Subject: [PATCH 005/105] Setup connection in the background, better drop table --- src/Interpreters/InterpreterDropQuery.cpp | 8 + .../PostgreSQLReplicationHandler.cpp | 70 ++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 14 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 140 ++++++++++-------- .../PostgreSQL/StoragePostgreSQLReplica.h | 9 +- .../test_storage_postgresql_replica/test.py | 27 ++++ 6 files changed, 177 insertions(+), 91 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index be4908582a5..68c9c2b41cf 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -21,6 +21,9 @@ # include #endif +#if USE_LIBPQXX +# include +#endif namespace DB { @@ -162,6 +165,11 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat table->shutdown(); +#if USE_LIBPQXX + if (table->getName() == "PostgreSQLReplica") + table->as()->shutdownFinal(); +#endif + TableExclusiveLockHolder table_lock; if (database->getUUID() == UUIDHelpers::Nil) table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f6571ce5a2e..5a68ef65925 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -20,17 +20,21 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static const auto reschedule_ms = 500; + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const std::string & table_name_, const std::string & conn_str, - const std::string & replication_slot_, - const std::string & publication_name_) + std::shared_ptr context_, + const std::string & publication_name_, + const std::string & replication_slot_name_) : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) + , context(context_) , database_name(database_name_) , table_name(table_name_) - , replication_slot(replication_slot_) , publication_name(publication_name_) + , replication_slot(replication_slot_name_) , connection(std::make_shared(conn_str)) { /// Create a replication connection, through which it is possible to execute only commands from streaming replication protocol @@ -38,39 +42,43 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( /// which will allow the connection to be used for logical replication from that database. replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); - /// Used commands require a specific transaction isolation mode. - replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); - /// Non temporary replication slot. Should be the same at restart. if (replication_slot.empty()) replication_slot = fmt::format("{}_{}_ch_replication_slot", database_name, table_name); /// Temporary replication slot is used to acquire a snapshot for initial table synchronization and to determine starting lsn position. temp_replication_slot = replication_slot + "_temp"; + + startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); + startup_task->deactivate(); } -void PostgreSQLReplicationHandler::startup(StoragePtr storage, std::shared_ptr context_) +void PostgreSQLReplicationHandler::startup(StoragePtr storage) { helper_table = storage; - context = context_; - tx = std::make_shared(*connection->conn()); - if (publication_name.empty()) - { - publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); + startup_task->activateAndSchedule(); +} - /// Publication defines what tables are included into replication stream. Should be deleted only if MaterializePostgreSQL - /// table is dropped. - if (!isPublicationExist()) - createPublication(); - } - else if (!isPublicationExist()) + +void PostgreSQLReplicationHandler::waitConnectionAndStart() +{ + try { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Publication name '{}' is spesified in table arguments, but it does not exist", publication_name); + /// Used commands require a specific transaction isolation mode. + replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); + } + catch (pqxx::broken_connection const & pqxx_error) + { + LOG_ERROR(log, "Unable to set up connection for table {}.{}. Reconnection attempt continues. Error message: {}", + database_name, table_name, pqxx_error.what()); + startup_task->scheduleAfter(reschedule_ms); + } + catch (Exception & e) + { + e.addMessage("while setting up connection for {}.{}", database_name, table_name); + throw; } - tx->commit(); startReplication(); } @@ -119,6 +127,24 @@ void PostgreSQLReplicationHandler::createPublication() void PostgreSQLReplicationHandler::startReplication() { + tx = std::make_shared(*connection->conn()); + if (publication_name.empty()) + { + publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); + + /// Publication defines what tables are included into replication stream. Should be deleted only if MaterializePostgreSQL + /// table is dropped. + if (!isPublicationExist()) + createPublication(); + } + else if (!isPublicationExist()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Publication name '{}' is spesified in table arguments, but it does not exist", publication_name); + } + tx->commit(); + auto ntx = std::make_shared(*replication_connection->conn()); /// Normally temporary replication slot should not exist. diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index af4465b863a..2e85bae5cb9 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -3,6 +3,7 @@ #include #include "PostgreSQLConnection.h" #include "PostgreSQLReplicaConsumer.h" +#include #include #include "pqxx/pqxx" @@ -21,16 +22,18 @@ public: const std::string & database_name_, const std::string & table_name_, const std::string & conn_str_, - const std::string & replication_slot_name_, - const std::string & publication_name_); + std::shared_ptr context_, + const std::string & publication_slot_name_, + const std::string & replication_slot_name_); - void startup(StoragePtr storage_, std::shared_ptr context_); + void startup(StoragePtr storage_); void shutdown(); void checkAndDropReplicationSlot(); private: using NontransactionPtr = std::shared_ptr; + void waitConnectionAndStart(); bool isPublicationExist(); void createPublication(); @@ -45,17 +48,18 @@ private: void getTableOutput(const Context & query_context); Poco::Logger * log; + std::shared_ptr context; const std::string database_name, table_name; - std::string replication_slot, publication_name; + std::string publication_name, replication_slot; std::string temp_replication_slot; PostgreSQLConnectionPtr connection; PostgreSQLConnectionPtr replication_connection; std::shared_ptr tx; + BackgroundSchedulePool::TaskHolder startup_task; std::shared_ptr consumer; - std::shared_ptr context; StoragePtr helper_table; //LSNPosition start_lsn, final_lsn; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 9ae90e24429..bca77f314cd 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include @@ -44,22 +45,30 @@ static auto nested_storage_suffix = "_ReplacingMergeTree"; StoragePostgreSQLReplica::StoragePostgreSQLReplica( const StorageID & table_id_, - const String & remote_table_name_, + const String & remote_database_name, + const String & remote_table_name, + const String & connection_str, const String & relative_data_path_, const StorageInMemoryMetadata & storage_metadata, const Context & context_, - const PostgreSQLReplicationHandler & replication_handler_, std::unique_ptr replication_settings_) : IStorage(table_id_) - , remote_table_name(remote_table_name_) , relative_data_path(relative_data_path_) - , global_context(std::make_shared(context_)) + , global_context(std::make_shared(context_.getGlobalContext())) , replication_settings(std::move(replication_settings_)) - , replication_handler(std::make_unique(replication_handler_)) { setInMemoryMetadata(storage_metadata); relative_data_path.resize(relative_data_path.size() - 1); relative_data_path += nested_storage_suffix; + + replication_handler = std::make_unique( + remote_database_name, + remote_table_name, + connection_str, + global_context, + global_context->getMacros()->expand(replication_settings->postgresql_replication_slot_name.value), + global_context->getMacros()->expand(replication_settings->postgresql_publication_name.value) + ); } @@ -144,47 +153,6 @@ ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() } -void StoragePostgreSQLReplica::startup() -{ - Context context_copy(*global_context); - const auto ast_create = getCreateHelperTableQuery(); - - Poco::File path(relative_data_path); - if (!path.exists()) - { - LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), - "Creating helper table {}", getStorageID().table_name + nested_storage_suffix); - InterpreterCreateQuery interpreter(ast_create, context_copy); - interpreter.execute(); - } - - nested_storage = createTableFromAST(ast_create->as(), getStorageID().database_name, relative_data_path, context_copy, false).second; - nested_storage->startup(); - - replication_handler->startup(nested_storage, global_context); -} - - -void StoragePostgreSQLReplica::drop() -{ - /// TODO: Under lock? Make sure synchronization stopped. - replication_handler->checkAndDropReplicationSlot(); - - nested_storage->drop(); - - relative_data_path.resize(relative_data_path.size() - 1); - Poco::File path(relative_data_path); - if (path.exists()) - path.remove(true); -} - - -void StoragePostgreSQLReplica::shutdown() -{ - replication_handler->shutdown(); -} - - Pipe StoragePostgreSQLReplica::read( const Names & column_names, const StorageMetadataPtr & /* metadata_snapshot */, @@ -208,6 +176,62 @@ Pipe StoragePostgreSQLReplica::read( } +void StoragePostgreSQLReplica::startup() +{ + Context context_copy(*global_context); + const auto ast_create = getCreateHelperTableQuery(); + + Poco::File path(relative_data_path); + if (!path.exists()) + { + LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), + "Creating helper table {}", getStorageID().table_name + nested_storage_suffix); + InterpreterCreateQuery interpreter(ast_create, context_copy); + interpreter.execute(); + } + else + LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), + "Directory already exists {}", relative_data_path); + + nested_storage = createTableFromAST(ast_create->as(), getStorageID().database_name, relative_data_path, context_copy, false).second; + nested_storage->startup(); + + replication_handler->startup(nested_storage); +} + + +void StoragePostgreSQLReplica::shutdown() +{ + replication_handler->shutdown(); +} + + +void StoragePostgreSQLReplica::shutdownFinal() +{ + /// TODO: Under lock? Make sure synchronization stopped. + replication_handler->checkAndDropReplicationSlot(); + dropNested(); +} + + +void StoragePostgreSQLReplica::dropNested() +{ + auto table_id = nested_storage->getStorageID(); + auto ast_drop = std::make_shared(); + + ast_drop->kind = ASTDropQuery::Drop; + ast_drop->table = table_id.table_name; + ast_drop->database = table_id.database_name; + ast_drop->if_exists = true; + + auto drop_context(*global_context); + drop_context.makeQueryContext(); + + auto interpreter = InterpreterDropQuery(ast_drop, drop_context); + interpreter.execute(); +} + + NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const { return NamesAndTypesList{}; @@ -252,22 +276,18 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) const String & remote_table = engine_args[2]->as().value.safeGet(); const String & remote_database = engine_args[1]->as().value.safeGet(); - String connection_str; - connection_str = fmt::format("dbname={} host={} port={} user={} password={}", - remote_database, - parsed_host_port.first, std::to_string(parsed_host_port.second), - engine_args[3]->as().value.safeGet(), - engine_args[4]->as().value.safeGet()); - - auto global_context(args.context.getGlobalContext()); - auto replication_slot_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_replication_slot_name.value); - auto publication_name = global_context.getMacros()->expand(postgresql_replication_settings->postgresql_publication_name.value); - - PostgreSQLReplicationHandler replication_handler(remote_database, remote_table, connection_str, replication_slot_name, publication_name); + /// No connection is made here, see Storages/PostgreSQL/PostgreSQLConnection.cpp + PostgreSQLConnection connection( + remote_database, + parsed_host_port.first, + parsed_host_port.second, + engine_args[3]->as().value.safeGet(), + engine_args[4]->as().value.safeGet()); return StoragePostgreSQLReplica::create( - args.table_id, remote_table, args.relative_data_path, metadata, global_context, - replication_handler, std::move(postgresql_replication_settings)); + args.table_id, remote_database, remote_table, connection.conn_str(), + args.relative_data_path, metadata, args.context, + std::move(postgresql_replication_settings)); }; factory.registerStorage( diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 18b0504d32c..c02c9696d87 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -33,7 +33,6 @@ public: String getName() const override { return "PostgreSQLReplica"; } void startup() override; - void drop() override; void shutdown() override; NamesAndTypesList getVirtuals() const override; @@ -47,15 +46,17 @@ public: size_t max_block_size, unsigned num_streams) override; + void shutdownFinal(); protected: StoragePostgreSQLReplica( const StorageID & table_id_, - const String & remote_table_name_, + const String & remote_database_name, + const String & remote_table_name, + const String & connection_str, const String & relative_data_path_, const StorageInMemoryMetadata & storage_metadata, const Context & context_, - const PostgreSQLReplicationHandler & replication_handler_, std::unique_ptr replication_settings_); private: @@ -64,8 +65,8 @@ private: std::shared_ptr getColumnsListFromStorage(); ASTPtr getColumnDeclaration(const DataTypePtr & data_type); ASTPtr getCreateHelperTableQuery(); + void dropNested(); - String remote_table_name; String relative_data_path; std::shared_ptr global_context; diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 3d98486fb4b..ccf0b2eee13 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -78,9 +78,36 @@ def test_initial_load_from_snapshot(started_cluster): ''') result = instance.query('SELECT * FROM test.postgresql_replica;') + cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) +def test_no_connection_at_startup(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + + started_cluster.pause_container('postgres1') + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + time.sleep(3) + started_cluster.unpause_container('postgres1') + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while int(result) == 0: + result = instance.query('SELECT count() FROM test.postgresql_replica;') + time.sleep(1); + print(result) + + result = instance.query('SELECT * FROM test.postgresql_replica;') + cursor.execute('DROP TABLE postgresql_replica;') + postgresql_replica_check_result(result, True) + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 4aadd0c3f218465589c9a36a285bccc88a9b8889 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 31 Jan 2021 19:03:03 +0000 Subject: [PATCH 006/105] Replicate insert queries --- .../PostgreSQLBlockInputStream.cpp | 195 +--------------- src/DataStreams/PostgreSQLBlockInputStream.h | 14 +- .../PostgreSQLReplicaBlockInputStream.cpp | 149 ------------- .../PostgreSQLReplicaBlockInputStream.h | 47 ---- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 148 +++++++++++-- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 40 +++- .../PostgreSQLReplicaConsumerBuffer.cpp | 38 ---- .../PostgreSQLReplicaConsumerBuffer.h | 39 ---- .../PostgreSQLReplicationHandler.cpp | 27 ++- .../PostgreSQL/PostgreSQLReplicationHandler.h | 9 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 20 +- .../PostgreSQL/StoragePostgreSQLReplica.h | 2 +- src/Storages/PostgreSQL/buffer_fwd.h | 9 - .../PostgreSQL/insertPostgreSQLValue.cpp | 208 ++++++++++++++++++ .../PostgreSQL/insertPostgreSQLValue.h | 29 +++ .../test_storage_postgresql_replica/test.py | 66 +++++- 16 files changed, 513 insertions(+), 527 deletions(-) delete mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp delete mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h delete mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp delete mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h delete mode 100644 src/Storages/PostgreSQL/buffer_fwd.h create mode 100644 src/Storages/PostgreSQL/insertPostgreSQLValue.cpp create mode 100644 src/Storages/PostgreSQL/insertPostgreSQLValue.h diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 4646a8a9d32..a52ca1e58a4 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -22,11 +22,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( std::unique_ptr tx_, const std::string & query_str_, @@ -38,8 +33,8 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( { description.init(sample_block); for (const auto idx : ext::range(0, description.sample_block.columns())) - if (description.types[idx].first == ValueType::vtArray) - prepareArrayInfo(idx, description.sample_block.getByPosition(idx).type); + if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) + preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); /// pqxx::stream_from uses COPY command, will get error if ';' is present if (query_str.ends_with(';')) query_str.resize(query_str.size() - 1); @@ -80,12 +75,17 @@ Block PostgreSQLBlockInputStream::readImpl() { ColumnNullable & column_nullable = assert_cast(*columns[idx]); const auto & data_type = assert_cast(*sample.type); - insertValue(column_nullable.getNestedColumn(), (*row)[idx], description.types[idx].first, data_type.getNestedType(), idx); + + insertPostgreSQLValue( + column_nullable.getNestedColumn(), (*row)[idx], + description.types[idx].first, data_type.getNestedType(), array_info, idx); + column_nullable.getNullMapData().emplace_back(0); } else { - insertValue(*columns[idx], (*row)[idx], description.types[idx].first, sample.type, idx); + insertPostgreSQLValue( + *columns[idx], (*row)[idx], description.types[idx].first, sample.type, array_info, idx); } } else @@ -113,183 +113,6 @@ void PostgreSQLBlockInputStream::readSuffix() } -void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view value, - const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx) -{ - switch (type) - { - case ValueType::vtUInt8: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtUInt16: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtUInt32: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtUInt64: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtInt8: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtInt16: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtInt32: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtInt64: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtFloat32: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtFloat64: - assert_cast(column).insertValue(pqxx::from_string(value)); - break; - case ValueType::vtFixedString:[[fallthrough]]; - case ValueType::vtString: - assert_cast(column).insertData(value.data(), value.size()); - break; - case ValueType::vtUUID: - assert_cast(column).insert(parse(value.data(), value.size())); - break; - case ValueType::vtDate: - assert_cast(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()}); - break; - case ValueType::vtDateTime: - assert_cast(column).insertValue(time_t{LocalDateTime{std::string(value)}}); - break; - case ValueType::vtDateTime64:[[fallthrough]]; - case ValueType::vtDecimal32: [[fallthrough]]; - case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128: [[fallthrough]]; - case ValueType::vtDecimal256: - { - ReadBufferFromString istr(value); - data_type->deserializeAsWholeText(column, istr, FormatSettings{}); - break; - } - case ValueType::vtArray: - { - pqxx::array_parser parser{value}; - std::pair parsed = parser.get_next(); - - size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions; - const auto parse_value = array_info[idx].pqxx_parser; - std::vector> dimensions(expected_dimensions + 1); - - while (parsed.first != pqxx::array_parser::juncture::done) - { - if ((parsed.first == pqxx::array_parser::juncture::row_start) && (++dimension > expected_dimensions)) - throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS); - - else if (parsed.first == pqxx::array_parser::juncture::string_value) - dimensions[dimension].emplace_back(parse_value(parsed.second)); - - else if (parsed.first == pqxx::array_parser::juncture::null_value) - dimensions[dimension].emplace_back(array_info[idx].default_value); - - else if (parsed.first == pqxx::array_parser::juncture::row_end) - { - max_dimension = std::max(max_dimension, dimension); - - if (--dimension == 0) - break; - - dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); - dimensions[dimension + 1].clear(); - } - - parsed = parser.get_next(); - } - - if (max_dimension < expected_dimensions) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Got less dimensions than expected. ({} instead of {})", max_dimension, expected_dimensions); - - assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); - break; - } - } -} - - -void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataTypePtr data_type) -{ - const auto * array_type = typeid_cast(data_type.get()); - auto nested = array_type->getNestedType(); - - size_t count_dimensions = 1; - while (isArray(nested)) - { - ++count_dimensions; - nested = typeid_cast(nested.get())->getNestedType(); - } - - Field default_value = nested->getDefault(); - if (nested->isNullable()) - nested = static_cast(nested.get())->getNestedType(); - - WhichDataType which(nested); - std::function parser; - - if (which.isUInt8() || which.isUInt16()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isInt8() || which.isInt16()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isUInt32()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isInt32()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isUInt64()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isInt64()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isFloat32()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isFloat64()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isString() || which.isFixedString()) - parser = [](std::string & field) -> Field { return field; }; - else if (which.isDate()) - parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; }; - else if (which.isDateTime()) - parser = [](std::string & field) -> Field { return time_t{LocalDateTime{field}}; }; - else if (which.isDecimal32()) - parser = [nested](std::string & field) -> Field - { - const auto & type = typeid_cast *>(nested.get()); - DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); - return convertFieldToType(field, res); - }; - else if (which.isDecimal64()) - parser = [nested](std::string & field) -> Field - { - const auto & type = typeid_cast *>(nested.get()); - DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); - return convertFieldToType(field, res); - }; - else if (which.isDecimal128()) - parser = [nested](std::string & field) -> Field - { - const auto & type = typeid_cast *>(nested.get()); - DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); - return convertFieldToType(field, res); - }; - else if (which.isDecimal256()) - parser = [nested](std::string & field) -> Field - { - const auto & type = typeid_cast *>(nested.get()); - DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); - return convertFieldToType(field, res); - }; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); - - array_info[column_idx] = {count_dimensions, default_value, parser}; -} } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 1e52b48c7cf..c18ccd0f55e 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -29,19 +30,14 @@ public: Block getHeader() const override { return description.sample_block.cloneEmpty(); } private: - using ValueType = ExternalResultDescription::ValueType; - void readPrefix() override; Block readImpl() override; void readSuffix() override; - void insertValue(IColumn & column, std::string_view value, - const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx); void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } - void prepareArrayInfo(size_t column_idx, const DataTypePtr data_type); String query_str; const UInt64 max_block_size; @@ -51,13 +47,7 @@ private: std::unique_ptr tx; std::unique_ptr stream; - struct ArrayInfo - { - size_t num_dimensions; - Field default_value; - std::function pqxx_parser; - }; - std::unordered_map array_info; + std::unordered_map array_info; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp deleted file mode 100644 index 04ee68eb3aa..00000000000 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.cpp +++ /dev/null @@ -1,149 +0,0 @@ -#include "PostgreSQLReplicaBlockInputStream.h" - -#include -#include -#include - - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -namespace DB -{ - -PostgreSQLReplicaBlockInputStream::PostgreSQLReplicaBlockInputStream( - StoragePostgreSQLReplica & storage_, - ConsumerBufferPtr buffer_, - const StorageMetadataPtr & metadata_snapshot_, - std::shared_ptr context_, - const Names & columns, - size_t max_block_size_) - : storage(storage_) - , buffer(buffer_) - , metadata_snapshot(metadata_snapshot_) - , context(context_) - , column_names(columns) - , max_block_size(max_block_size_) - , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) - , sample_block(non_virtual_header) - , virtual_header(metadata_snapshot->getSampleBlockForColumns({}, storage.getVirtuals(), storage.getStorageID())) -{ - for (const auto & column : virtual_header) - sample_block.insert(column); -} - - -PostgreSQLReplicaBlockInputStream::~PostgreSQLReplicaBlockInputStream() -{ -} - - -void PostgreSQLReplicaBlockInputStream::readPrefixImpl() -{ -} - - -Block PostgreSQLReplicaBlockInputStream::readImpl() -{ - if (!buffer || finished) - return Block(); - - finished = true; - - MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); - MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - - auto input_format = FormatFactory::instance().getInputFormat( - "Values", *buffer, non_virtual_header, *context, max_block_size); - - InputPort port(input_format->getPort().getHeader(), input_format.get()); - connect(input_format->getPort(), port); - port.setNeeded(); - - auto read_rabbitmq_message = [&] - { - size_t new_rows = 0; - - while (true) - { - auto status = input_format->prepare(); - - switch (status) - { - case IProcessor::Status::Ready: - input_format->work(); - break; - - case IProcessor::Status::Finished: - input_format->resetParser(); - return new_rows; - - case IProcessor::Status::PortFull: - { - auto chunk = port.pull(); - - auto chunk_rows = chunk.getNumRows(); - new_rows += chunk_rows; - - auto columns = chunk.detachColumns(); - - for (size_t i = 0, s = columns.size(); i < s; ++i) - { - result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - } - break; - } - case IProcessor::Status::NeedData: - case IProcessor::Status::Async: - case IProcessor::Status::ExpandPipeline: - throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); - } - } - }; - - size_t total_rows = 0; - - while (true) - { - if (buffer->eof()) - break; - - auto new_rows = read_rabbitmq_message(); - - if (new_rows) - { - //auto timestamp = buffer->getTimestamp(); - //for (size_t i = 0; i < new_rows; ++i) - //{ - // virtual_columns[0]->insert(timestamp); - //} - - total_rows = total_rows + new_rows; - } - - buffer->allowNext(); - - if (total_rows >= max_block_size || !checkTimeLimit()) - break; - } - - if (total_rows == 0) - return Block(); - - auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); - auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - - for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - result_block.insert(column); - - return result_block; -} - - -void PostgreSQLReplicaBlockInputStream::readSuffixImpl() -{ -} - -} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h b/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h deleted file mode 100644 index 995c640682a..00000000000 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaBlockInputStream.h +++ /dev/null @@ -1,47 +0,0 @@ -#pragma once - -#include -#include "StoragePostgreSQLReplica.h" -#include "PostgreSQLReplicaConsumerBuffer.h" -#include "buffer_fwd.h" - - -namespace DB -{ - -class PostgreSQLReplicaBlockInputStream : public IBlockInputStream -{ - -public: - PostgreSQLReplicaBlockInputStream( - StoragePostgreSQLReplica & storage_, - ConsumerBufferPtr buffer_, - const StorageMetadataPtr & metadata_snapshot_, - std::shared_ptr context_, - const Names & columns, - size_t max_block_size_); - - ~PostgreSQLReplicaBlockInputStream() override; - - String getName() const override { return storage.getName(); } - Block getHeader() const override { return sample_block; } - - void readPrefixImpl() override; - Block readImpl() override; - void readSuffixImpl() override; - -private: - StoragePostgreSQLReplica & storage; - ConsumerBufferPtr buffer; - StorageMetadataPtr metadata_snapshot; - std::shared_ptr context; - Names column_names; - const size_t max_block_size; - - bool finished = false; - const Block non_virtual_header; - Block sample_block; - const Block virtual_header; -}; - -} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 076863eb8dd..c38b898fdc1 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -2,13 +2,30 @@ #include #include + #include + #include #include #include + #include #include + #include +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include +#include namespace DB { @@ -17,9 +34,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static const auto wal_reader_reschedule_ms = 500; +static const auto reschedule_ms = 500; static const auto max_thread_work_duration_ms = 60000; -static const auto max_empty_slot_reads = 20; +static const auto max_empty_slot_reads = 2; PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( std::shared_ptr context_, @@ -27,7 +44,9 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( const std::string & conn_str, const std::string & replication_slot_name_, const std::string & publication_name_, - const LSNPosition & start_lsn) + const LSNPosition & start_lsn, + const size_t max_block_size_, + StoragePtr nested_storage_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) , context(context_) , replication_slot_name(replication_slot_name_) @@ -35,40 +54,49 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( , table_name(table_name_) , connection(std::make_shared(conn_str)) , current_lsn(start_lsn) + , max_block_size(max_block_size_) + , nested_storage(nested_storage_) + , sample_block(nested_storage->getInMemoryMetadata().getSampleBlock()) { replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); - wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ WALReaderFunc(); }); + description.init(sample_block); + for (const auto idx : ext::range(0, description.sample_block.columns())) + if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) + preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); + + columns = description.sample_block.cloneEmptyColumns(); + + wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ replicationStream(); }); wal_reader_task->deactivate(); } void PostgreSQLReplicaConsumer::startSynchronization() { - //wal_reader_task->activateAndSchedule(); + wal_reader_task->activateAndSchedule(); } void PostgreSQLReplicaConsumer::stopSynchronization() { stop_synchronization.store(true); - if (wal_reader_task) - wal_reader_task->deactivate(); + wal_reader_task->deactivate(); } -void PostgreSQLReplicaConsumer::WALReaderFunc() +void PostgreSQLReplicaConsumer::replicationStream() { size_t count_empty_slot_reads = 0; auto start_time = std::chrono::steady_clock::now(); - LOG_TRACE(log, "Starting synchronization thread"); + LOG_TRACE(log, "Starting replication stream"); while (!stop_synchronization) { if (!readFromReplicationSlot() && ++count_empty_slot_reads == max_empty_slot_reads) { - LOG_TRACE(log, "Reschedule synchronization. Replication slot is empty."); + LOG_TRACE(log, "Reschedule replication stream. Replication slot is empty."); break; } else @@ -78,13 +106,38 @@ void PostgreSQLReplicaConsumer::WALReaderFunc() auto duration = std::chrono::duration_cast(end_time - start_time); if (duration.count() > max_thread_work_duration_ms) { - LOG_TRACE(log, "Reschedule synchronization. Thread work duration limit exceeded."); + LOG_TRACE(log, "Reschedule replication_stream. Thread work duration limit exceeded."); break; } } if (!stop_synchronization) - wal_reader_task->scheduleAfter(wal_reader_reschedule_ms); + wal_reader_task->scheduleAfter(reschedule_ms); +} + + +void PostgreSQLReplicaConsumer::insertValue(std::string & value, size_t column_idx) +{ + const auto & sample = description.sample_block.getByPosition(column_idx); + bool is_nullable = description.types[column_idx].second; + + LOG_TRACE(log, "INSERTING VALUE {}", value); + if (is_nullable) + { + ColumnNullable & column_nullable = assert_cast(*columns[column_idx]); + const auto & data_type = assert_cast(*sample.type); + + insertPostgreSQLValue( + column_nullable.getNestedColumn(), value, + description.types[column_idx].first, data_type.getNestedType(), array_info, column_idx); + + column_nullable.getNullMapData().emplace_back(0); + } + else + { + insertPostgreSQLValue( + *columns[column_idx], value, description.types[column_idx].first, sample.type, array_info, column_idx); + } } @@ -150,17 +203,24 @@ void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos Int16 num_columns = readInt16(message, pos); /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data LOG_DEBUG(log, "num_columns {}", num_columns); - for (int k = 0; k < num_columns; ++k) + for (int column_idx = 0; column_idx < num_columns; ++column_idx) { char identifier = readInt8(message, pos); Int32 col_len = readInt32(message, pos); - String result; + String value; for (int i = 0; i < col_len; ++i) { - result += readInt8(message, pos); + value += readInt8(message, pos); } - LOG_DEBUG(log, "identifier {}, col_len {}, result {}", identifier, col_len, result); + + insertValue(value, column_idx); + + LOG_DEBUG(log, "identifier {}, col_len {}, value {}", identifier, col_len, value); } + + String val = "1"; + insertValue(val, num_columns); + insertValue(val, num_columns + 1); //readString(message, pos, size, result); } @@ -171,6 +231,7 @@ void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replicatio size_t pos = 2; char type = readInt8(replication_message, pos); + LOG_TRACE(log, "TYPE: {}", type); switch (type) { @@ -180,6 +241,7 @@ void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replicatio Int64 transaction_commit_timestamp = readInt64(replication_message, pos); LOG_DEBUG(log, "transaction lsn {}, transaction commit timespamp {}", transaction_end_lsn, transaction_commit_timestamp); + //current_lsn.lsn_value = transaction_end_lsn; break; } case 'C': // Commit @@ -191,6 +253,7 @@ void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replicatio Int64 transaction_commit_timestamp = readInt64(replication_message, pos); LOG_DEBUG(log, "commit lsn {}, transaction lsn {}, transaction commit timestamp {}", commit_lsn, transaction_end_lsn, transaction_commit_timestamp); + final_lsn.lsn = current_lsn.lsn; break; } case 'O': // Origin @@ -245,16 +308,49 @@ void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replicatio } +void PostgreSQLReplicaConsumer::syncIntoTable(Block & block) +{ + Context insert_context(*context); + insert_context.makeQueryContext(); + + auto insert = std::make_shared(); + insert->table_id = nested_storage->getStorageID(); + + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); + OneBlockInputStream input(block); + + copyData(input, *block_io.out); + LOG_TRACE(log, "TABLE SYNC END"); +} + + +void PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr ntx) +{ + LOG_TRACE(log, "CURRENT LSN FROM TO {}", final_lsn.lsn); + std::string query_str = fmt::format("SELECT pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn.lsn); + pqxx::result result{ntx->exec(query_str)}; + if (!result.empty()) + { + std::string s1 = result[0].size() > 0 && !result[0][0].is_null() ? result[0][0].as() : "NULL"; + std::string s2 = result[0].size() > 1 && !result[0][1].is_null() ? result[0][1].as() : "NULL"; + LOG_TRACE(log, "ADVANCE LSN: {} and {}", s1, s2); + + } +} + + /// Read binary changes from replication slot via copy command. bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { + columns = description.sample_block.cloneEmptyColumns(); bool slot_empty = true; try { - auto tx = std::make_unique(*replication_connection->conn()); + auto tx = std::make_shared(*replication_connection->conn()); /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. std::string query_str = fmt::format( - "select data FROM pg_logical_slot_peek_binary_changes(" + "select lsn, data FROM pg_logical_slot_peek_binary_changes(" "'{}', NULL, NULL, 'publication_names', '{}', 'proto_version', '1')", replication_slot_name, publication_name); pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); @@ -267,17 +363,23 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { LOG_TRACE(log, "STREAM REPLICATION END"); stream.complete(); + + Block result_rows = description.sample_block.cloneWithColumns(std::move(columns)); + if (result_rows.rows()) + { + syncIntoTable(result_rows); + advanceLSN(tx); + } + tx->commit(); break; } slot_empty = false; - for (const auto idx : ext::range(0, row->size())) - { - LOG_TRACE(log, "Replication message: {}", (*row)[idx]); - decodeReplicationMessage((*row)[idx].c_str(), (*row)[idx].size()); - } + current_lsn.lsn = (*row)[0]; + LOG_TRACE(log, "Replication message: {}", (*row)[1]); + decodeReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } } catch (...) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index bb4b4c5033b..ca357236180 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -3,7 +3,10 @@ #include "PostgreSQLConnection.h" #include #include +#include +#include #include "pqxx/pqxx" +#include namespace DB { @@ -11,8 +14,9 @@ namespace DB struct LSNPosition { std::string lsn; + int64_t lsn_value; - uint64_t getValue() + int64_t getValue() { uint64_t upper_half, lower_half, result; std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); @@ -22,6 +26,15 @@ struct LSNPosition // upper_half, lower_half, result); return result; } + + std::string getString() + { + char result[16]; + std::snprintf(result, sizeof(result), "%lX/%lX", (lsn_value >> 32), lsn_value & 0xFFFFFFFF); + //assert(lsn_value == result.getValue()); + std::string ans = result; + return ans; + } }; @@ -34,20 +47,28 @@ public: const std::string & conn_str_, const std::string & replication_slot_name_, const std::string & publication_name_, - const LSNPosition & start_lsn); + const LSNPosition & start_lsn, + const size_t max_block_size_, + StoragePtr nested_storage_); /// Start reading WAL from current_lsn position. Initial data sync from created snapshot already done. void startSynchronization(); void stopSynchronization(); private: - /// Executed by wal_reader_task. A separate thread reads wal and advances lsn when rows were written via copyData. - void WALReaderFunc(); + /// Executed by wal_reader_task. A separate thread reads wal and advances lsn to last commited position + /// after rows were written via copyData. + void replicationStream(); + void stopReplicationStream(); /// Start changes stream from WAL via copy command (up to max_block_size changes). bool readFromReplicationSlot(); void decodeReplicationMessage(const char * replication_message, size_t size); + void insertValue(std::string & value, size_t column_idx); + void syncIntoTable(Block & block); + void advanceLSN(std::shared_ptr ntx); + /// Methods to parse replication message data. void readTupleData(const char * message, size_t & pos, size_t size); void readString(const char * message, size_t & pos, size_t size, String & result); @@ -64,9 +85,18 @@ private: const std::string table_name; PostgreSQLConnectionPtr connection, replication_connection; - LSNPosition current_lsn; + LSNPosition current_lsn, final_lsn; BackgroundSchedulePool::TaskHolder wal_reader_task; + //BackgroundSchedulePool::TaskHolder table_sync_task; std::atomic stop_synchronization = false; + + const size_t max_block_size; + StoragePtr nested_storage; + Block sample_block; + ExternalResultDescription description; + MutableColumns columns; + /// Needed for insertPostgreSQLValue() method to parse array + std::unordered_map array_info; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp deleted file mode 100644 index e8c4ba3d55a..00000000000 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.cpp +++ /dev/null @@ -1,38 +0,0 @@ -#include "PostgreSQLReplicaConsumerBuffer.h" - - -namespace DB -{ - -PostgreSQLReplicaConsumerBuffer::PostgreSQLReplicaConsumerBuffer( - uint64_t max_block_size_) - : ReadBuffer(nullptr, 0) - , rows_data(max_block_size_) -{ -} - - -PostgreSQLReplicaConsumerBuffer::~PostgreSQLReplicaConsumerBuffer() -{ - BufferBase::set(nullptr, 0, 0); -} - - -bool PostgreSQLReplicaConsumerBuffer::nextImpl() -{ - if (!allowed) - return false; - - if (rows_data.tryPop(current_row_data)) - { - auto * new_position = const_cast(current_row_data.data.data()); - BufferBase::set(new_position, current_row_data.data.size(), 0); - allowed = false; - - return true; - } - - return false; -} - -} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h deleted file mode 100644 index 8c8de3a8b68..00000000000 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumerBuffer.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include "buffer_fwd.h" - - -namespace DB -{ - -class PostgreSQLReplicaConsumerBuffer : public ReadBuffer -{ - -public: - PostgreSQLReplicaConsumerBuffer( - uint64_t max_block_size_); - - ~PostgreSQLReplicaConsumerBuffer() override; - - void allowNext() { allowed = true; } - -private: - bool nextImpl() override; - - struct RowData - { - String data; - RowData() : data("") {} - }; - - RowData current_row_data; - ConcurrentBoundedQueue rows_data; - bool allowed = true; -}; - -} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 5a68ef65925..7b75c42c7a8 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -28,13 +28,15 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & conn_str, std::shared_ptr context_, const std::string & publication_name_, - const std::string & replication_slot_name_) + const std::string & replication_slot_name_, + const size_t max_block_size_) : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) , context(context_) , database_name(database_name_) , table_name(table_name_) , publication_name(publication_name_) , replication_slot(replication_slot_name_) + , max_block_size(max_block_size_) , connection(std::make_shared(conn_str)) { /// Create a replication connection, through which it is possible to execute only commands from streaming replication protocol @@ -56,7 +58,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( void PostgreSQLReplicationHandler::startup(StoragePtr storage) { - helper_table = storage; + nested_storage = storage; startup_task->activateAndSchedule(); } @@ -98,6 +100,7 @@ bool PostgreSQLReplicationHandler::isPublicationExist() assert(!result.empty()); bool publication_exists = (result[0][0].as() == "t"); + /// TODO: check if publication is still valid? if (publication_exists) LOG_TRACE(log, "Publication {} already exists. Using existing version", publication_name); @@ -121,7 +124,7 @@ void PostgreSQLReplicationHandler::createPublication() /// TODO: check replica identity /// Requires changed replica identity for included table to be able to receive old values of updated rows. - /// (ALTER TABLE table_name REPLICA IDENTITY FULL) + /// (ALTER TABLE table_name REPLICA IDENTITY FULL ?) } @@ -173,7 +176,9 @@ void PostgreSQLReplicationHandler::startReplication() connection->conn_str(), replication_slot, publication_name, - start_lsn); + start_lsn, + max_block_size, + nested_storage); LOG_DEBUG(log, "Commiting replication transaction"); ntx->commit(); @@ -203,12 +208,12 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) insert_context.makeQueryContext(); auto insert = std::make_shared(); - insert->table_id = helper_table->getStorageID(); + insert->table_id = nested_storage->getStorageID(); InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); - const StorageInMemoryMetadata & storage_metadata = helper_table->getInMemoryMetadata(); + const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); PostgreSQLBlockInputStream input(std::move(stx), query_str, sample_block, DEFAULT_BLOCK_SIZE); @@ -296,10 +301,18 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, st } +void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr ntx) +{ + std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); + ntx->exec(query_str); +} + + /// Only used when MaterializePostgreSQL table is dropped. -void PostgreSQLReplicationHandler::checkAndDropReplicationSlot() +void PostgreSQLReplicationHandler::removeSlotAndPublication() { auto ntx = std::make_shared(*replication_connection->conn()); + dropPublication(ntx); if (isReplicationSlotExist(ntx, replication_slot)) dropReplicationSlot(ntx, replication_slot, false); ntx->commit(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 2e85bae5cb9..5cc4d336921 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -24,11 +24,12 @@ public: const std::string & conn_str_, std::shared_ptr context_, const std::string & publication_slot_name_, - const std::string & replication_slot_name_); + const std::string & replication_slot_name_, + const size_t max_block_size_); void startup(StoragePtr storage_); void shutdown(); - void checkAndDropReplicationSlot(); + void removeSlotAndPublication(); private: using NontransactionPtr = std::shared_ptr; @@ -41,6 +42,7 @@ private: void createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name); void createReplicationSlot(NontransactionPtr ntx); void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name, bool use_replication_api); + void dropPublication(NontransactionPtr ntx); void startReplication(); void loadFromSnapshot(std::string & snapshot_name); @@ -53,6 +55,7 @@ private: std::string publication_name, replication_slot; std::string temp_replication_slot; + const size_t max_block_size; PostgreSQLConnectionPtr connection; PostgreSQLConnectionPtr replication_connection; @@ -60,7 +63,7 @@ private: BackgroundSchedulePool::TaskHolder startup_task; std::shared_ptr consumer; - StoragePtr helper_table; + StoragePtr nested_storage; //LSNPosition start_lsn, final_lsn; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index bca77f314cd..13cd5321737 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -27,7 +27,6 @@ #include #include "PostgreSQLReplicationSettings.h" -#include "PostgreSQLReplicaBlockInputStream.h" #include #include @@ -61,13 +60,15 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( relative_data_path.resize(relative_data_path.size() - 1); relative_data_path += nested_storage_suffix; + replication_handler = std::make_unique( remote_database_name, remote_table_name, connection_str, global_context, global_context->getMacros()->expand(replication_settings->postgresql_replication_slot_name.value), - global_context->getMacros()->expand(replication_settings->postgresql_publication_name.value) + global_context->getMacros()->expand(replication_settings->postgresql_publication_name.value), + global_context->getSettingsRef().postgresql_replica_max_rows_to_insert.value ); } @@ -180,12 +181,13 @@ void StoragePostgreSQLReplica::startup() { Context context_copy(*global_context); const auto ast_create = getCreateHelperTableQuery(); + auto table_id = getStorageID(); Poco::File path(relative_data_path); if (!path.exists()) { LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), - "Creating helper table {}", getStorageID().table_name + nested_storage_suffix); + "Creating helper table {}", table_id.table_name + nested_storage_suffix); InterpreterCreateQuery interpreter(ast_create, context_copy); interpreter.execute(); } @@ -193,8 +195,13 @@ void StoragePostgreSQLReplica::startup() LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), "Directory already exists {}", relative_data_path); - nested_storage = createTableFromAST(ast_create->as(), getStorageID().database_name, relative_data_path, context_copy, false).second; - nested_storage->startup(); + nested_storage = DatabaseCatalog::instance().getTable( + StorageID(table_id.database_name, table_id.table_name + nested_storage_suffix), + *global_context); + + //nested_storage = createTableFromAST( + // ast_create->as(), getStorageID().database_name, relative_data_path, context_copy, false).second; + //nested_storage->startup(); replication_handler->startup(nested_storage); } @@ -208,8 +215,7 @@ void StoragePostgreSQLReplica::shutdown() void StoragePostgreSQLReplica::shutdownFinal() { - /// TODO: Under lock? Make sure synchronization stopped. - replication_handler->checkAndDropReplicationSlot(); + replication_handler->removeSlotAndPublication(); dropNested(); } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index c02c9696d87..8dbfeb79bf0 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -19,7 +19,6 @@ #include #include "PostgreSQLReplicationHandler.h" #include "PostgreSQLReplicationSettings.h" -#include "buffer_fwd.h" #include "pqxx/pqxx" namespace DB @@ -46,6 +45,7 @@ public: size_t max_block_size, unsigned num_streams) override; + /// Called right after shutdown() in case of drop query void shutdownFinal(); protected: diff --git a/src/Storages/PostgreSQL/buffer_fwd.h b/src/Storages/PostgreSQL/buffer_fwd.h deleted file mode 100644 index 40ffd64aad3..00000000000 --- a/src/Storages/PostgreSQL/buffer_fwd.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once - -namespace DB -{ - -class PostgreSQLReplicaConsumerBuffer; -using ConsumerBufferPtr = std::shared_ptr; - -} diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp b/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp new file mode 100644 index 00000000000..5d4723364dc --- /dev/null +++ b/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp @@ -0,0 +1,208 @@ +#include "insertPostgreSQLValue.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +void insertPostgreSQLValue( + IColumn & column, std::string_view value, + const ExternalResultDescription::ValueType type, const DataTypePtr data_type, + std::unordered_map & array_info, size_t idx) +{ + switch (type) + { + case ExternalResultDescription::ValueType::vtUInt8: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtUInt16: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtUInt32: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtUInt64: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtInt8: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtInt16: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtInt32: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtInt64: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtFloat32: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtFloat64: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ExternalResultDescription::ValueType::vtFixedString:[[fallthrough]]; + case ExternalResultDescription::ValueType::vtString: + assert_cast(column).insertData(value.data(), value.size()); + break; + case ExternalResultDescription::ValueType::vtUUID: + assert_cast(column).insert(parse(value.data(), value.size())); + break; + case ExternalResultDescription::ValueType::vtDate: + assert_cast(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()}); + break; + case ExternalResultDescription::ValueType::vtDateTime: + assert_cast(column).insertValue(time_t{LocalDateTime{std::string(value)}}); + break; + case ExternalResultDescription::ValueType::vtDateTime64:[[fallthrough]]; + case ExternalResultDescription::ValueType::vtDecimal32: [[fallthrough]]; + case ExternalResultDescription::ValueType::vtDecimal64: [[fallthrough]]; + case ExternalResultDescription::ValueType::vtDecimal128: [[fallthrough]]; + case ExternalResultDescription::ValueType::vtDecimal256: + { + ReadBufferFromString istr(value); + data_type->deserializeAsWholeText(column, istr, FormatSettings{}); + break; + } + case ExternalResultDescription::ValueType::vtArray: + { + pqxx::array_parser parser{value}; + std::pair parsed = parser.get_next(); + + size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions; + const auto parse_value = array_info[idx].pqxx_parser; + std::vector> dimensions(expected_dimensions + 1); + + while (parsed.first != pqxx::array_parser::juncture::done) + { + if ((parsed.first == pqxx::array_parser::juncture::row_start) && (++dimension > expected_dimensions)) + throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS); + + else if (parsed.first == pqxx::array_parser::juncture::string_value) + dimensions[dimension].emplace_back(parse_value(parsed.second)); + + else if (parsed.first == pqxx::array_parser::juncture::null_value) + dimensions[dimension].emplace_back(array_info[idx].default_value); + + else if (parsed.first == pqxx::array_parser::juncture::row_end) + { + max_dimension = std::max(max_dimension, dimension); + + if (--dimension == 0) + break; + + dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); + dimensions[dimension + 1].clear(); + } + + parsed = parser.get_next(); + } + + if (max_dimension < expected_dimensions) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Got less dimensions than expected. ({} instead of {})", max_dimension, expected_dimensions); + + assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); + break; + } + } +} + + +void preparePostgreSQLArrayInfo( + std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type) +{ + const auto * array_type = typeid_cast(data_type.get()); + auto nested = array_type->getNestedType(); + + size_t count_dimensions = 1; + while (isArray(nested)) + { + ++count_dimensions; + nested = typeid_cast(nested.get())->getNestedType(); + } + + Field default_value = nested->getDefault(); + if (nested->isNullable()) + nested = static_cast(nested.get())->getNestedType(); + + WhichDataType which(nested); + std::function parser; + + if (which.isUInt8() || which.isUInt16()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isInt8() || which.isInt16()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUInt32()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isInt32()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUInt64()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isInt64()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isFloat32()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isFloat64()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isString() || which.isFixedString()) + parser = [](std::string & field) -> Field { return field; }; + else if (which.isDate()) + parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; }; + else if (which.isDateTime()) + parser = [](std::string & field) -> Field { return time_t{LocalDateTime{field}}; }; + else if (which.isDecimal32()) + parser = [nested](std::string & field) -> Field + { + const auto & type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else if (which.isDecimal64()) + parser = [nested](std::string & field) -> Field + { + const auto & type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else if (which.isDecimal128()) + parser = [nested](std::string & field) -> Field + { + const auto & type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else if (which.isDecimal256()) + parser = [nested](std::string & field) -> Field + { + const auto & type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); + + array_info[column_idx] = {count_dimensions, default_value, parser}; +} +} + diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.h b/src/Storages/PostgreSQL/insertPostgreSQLValue.h new file mode 100644 index 00000000000..1582d35d096 --- /dev/null +++ b/src/Storages/PostgreSQL/insertPostgreSQLValue.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct PostgreSQLArrayInfo +{ + size_t num_dimensions; + Field default_value; + std::function pqxx_parser; +}; + + +void insertPostgreSQLValue( + IColumn & column, std::string_view value, + const ExternalResultDescription::ValueType type, const DataTypePtr data_type, + std::unordered_map & array_info, size_t idx); + +void preparePostgreSQLArrayInfo( + std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type); + +} diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index ccf0b2eee13..487ee2a35cb 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -13,7 +13,7 @@ instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml' postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( - key Integer NOT NULL, value Integer, PRIMARY KEY (key)) + key Integer NOT NULL, value Integer) """ def get_postgres_conn(database=False): @@ -108,6 +108,70 @@ def test_no_connection_at_startup(started_cluster): cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) + +def test_detach_attach_is_ok(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + result = instance.query('SELECT * FROM test.postgresql_replica;') + postgresql_replica_check_result(result, True) + + instance.query('DETACH TABLE test.postgresql_replica') + instance.query('ATTACH TABLE test.postgresql_replica') + + result = instance.query('SELECT * FROM test.postgresql_replica;') + cursor.execute('DROP TABLE postgresql_replica;') + postgresql_replica_check_result(result, True) + + +def test_replicating_inserts(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 10) + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)") + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)") + + time.sleep(4) + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 30) + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)") + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)") + + time.sleep(4) + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 50) + + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + + cursor.execute('DROP TABLE postgresql_replica;') + postgresql_replica_check_result(result, True) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 0d19992128abc82cfa0cda9fee3abf939c8809ec Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Feb 2021 16:13:18 +0000 Subject: [PATCH 007/105] Replicate delete queries --- .../PostgreSQLBlockInputStream.cpp | 2 +- src/DataStreams/PostgreSQLBlockInputStream.h | 7 -- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 88 +++++++++++++++---- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 19 +++- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 75 ++++++++++++++++ .../PostgreSQL/PostgreSQLReplicaMetadata.h | 31 +++++++ .../PostgreSQLReplicationHandler.cpp | 19 +++- .../PostgreSQL/PostgreSQLReplicationHandler.h | 5 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 75 +++++++++++++--- .../PostgreSQL/StoragePostgreSQLReplica.h | 2 +- .../PostgreSQL/insertPostgreSQLValue.cpp | 8 ++ .../PostgreSQL/insertPostgreSQLValue.h | 2 + src/Storages/StoragePostgreSQL.h | 1 + .../test_storage_postgresql_replica/test.py | 58 +++++++++--- 14 files changed, 333 insertions(+), 59 deletions(-) create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index a52ca1e58a4..5b43a21c6fc 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -90,7 +90,7 @@ Block PostgreSQLBlockInputStream::readImpl() } else { - insertDefaultValue(*columns[idx], *sample.column); + insertDefaultPostgreSQLValue(*columns[idx], *sample.column); } } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index c18ccd0f55e..f51526b2eb3 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -15,7 +15,6 @@ namespace DB { -using ConnectionPtr = std::shared_ptr; class PostgreSQLBlockInputStream : public IBlockInputStream { @@ -34,16 +33,10 @@ private: Block readImpl() override; void readSuffix() override; - void insertDefaultValue(IColumn & column, const IColumn & sample_column) - { - column.insertFrom(sample_column, 0); - } - String query_str; const UInt64 max_block_size; ExternalResultDescription description; - ConnectionPtr connection; std::unique_ptr tx; std::unique_ptr stream; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index c38b898fdc1..3a91e893392 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -29,6 +29,7 @@ namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -36,7 +37,8 @@ namespace ErrorCodes static const auto reschedule_ms = 500; static const auto max_thread_work_duration_ms = 60000; -static const auto max_empty_slot_reads = 2; +static const auto max_empty_slot_reads = 16; + PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( std::shared_ptr context_, @@ -44,6 +46,7 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( const std::string & conn_str, const std::string & replication_slot_name_, const std::string & publication_name_, + const std::string & metadata_path, const LSNPosition & start_lsn, const size_t max_block_size_, StoragePtr nested_storage_) @@ -51,6 +54,7 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( , context(context_) , replication_slot_name(replication_slot_name_) , publication_name(publication_name_) + , metadata(metadata_path) , table_name(table_name_) , connection(std::make_shared(conn_str)) , current_lsn(start_lsn) @@ -69,6 +73,7 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ replicationStream(); }); wal_reader_task->deactivate(); + } @@ -118,10 +123,10 @@ void PostgreSQLReplicaConsumer::replicationStream() void PostgreSQLReplicaConsumer::insertValue(std::string & value, size_t column_idx) { + LOG_TRACE(log, "INSERTING VALUE {}", value); const auto & sample = description.sample_block.getByPosition(column_idx); bool is_nullable = description.types[column_idx].second; - LOG_TRACE(log, "INSERTING VALUE {}", value); if (is_nullable) { ColumnNullable & column_nullable = assert_cast(*columns[column_idx]); @@ -141,6 +146,13 @@ void PostgreSQLReplicaConsumer::insertValue(std::string & value, size_t column_i } +void PostgreSQLReplicaConsumer::insertDefaultValue(size_t column_idx) +{ + const auto & sample = description.sample_block.getByPosition(column_idx); + insertDefaultPostgreSQLValue(*columns[column_idx], *sample.column); +} + + void PostgreSQLReplicaConsumer::readString(const char * message, size_t & pos, size_t size, String & result) { assert(size > pos + 2); @@ -198,7 +210,7 @@ Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos) } -void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos, size_t /* size */) +void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos, PostgreSQLQuery type) { Int16 num_columns = readInt16(message, pos); /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data @@ -218,20 +230,36 @@ void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos LOG_DEBUG(log, "identifier {}, col_len {}, value {}", identifier, col_len, value); } - String val = "1"; - insertValue(val, num_columns); - insertValue(val, num_columns + 1); - //readString(message, pos, size, result); + switch (type) + { + case PostgreSQLQuery::INSERT: + { + columns[num_columns]->insert(Int8(1)); + columns[num_columns + 1]->insert(UInt64(metadata.version())); + //insertValueMaterialized(*columns[num_columns], 1); + //insertValueMaterialized(*columns[num_columns + 1], metadata.version()); + break; + } + case PostgreSQLQuery::DELETE: + { + columns[num_columns]->insert(Int8(-1)); + columns[num_columns + 1]->insert(UInt64(metadata.version())); + break; + } + case PostgreSQLQuery::UPDATE: + { + break; + } + } } -void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replication_message, size_t size) +void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replication_message, size_t size) { /// Skip '\x' size_t pos = 2; char type = readInt8(replication_message, pos); - LOG_TRACE(log, "TYPE: {}", type); switch (type) { @@ -292,13 +320,23 @@ void PostgreSQLReplicaConsumer::decodeReplicationMessage(const char * replicatio Int32 relation_id = readInt32(replication_message, pos); Int8 new_tuple = readInt8(replication_message, pos); LOG_DEBUG(log, "relationID {}, newTuple {}", relation_id, new_tuple); - readTupleData(replication_message, pos, size); + readTupleData(replication_message, pos, PostgreSQLQuery::INSERT); break; } case 'U': // Update break; case 'D': // Delete + { + Int32 relation_id = readInt32(replication_message, pos); + //Int8 index_replica_identity = readInt8(replication_message, pos); + Int8 full_replica_identity = readInt8(replication_message, pos); + LOG_DEBUG(log, "relationID {}, full replica identity {}", + relation_id, full_replica_identity); + //LOG_DEBUG(log, "relationID {}, index replica identity {} full replica identity {}", + // relation_id, index_replica_identity, full_replica_identity); + readTupleData(replication_message, pos, PostgreSQLQuery::DELETE); break; + } case 'T': // Truncate break; default: @@ -344,16 +382,18 @@ void PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { columns = description.sample_block.cloneEmptyColumns(); + std::shared_ptr tx; bool slot_empty = true; try { - auto tx = std::make_shared(*replication_connection->conn()); + tx = std::make_shared(*replication_connection->conn()); /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. std::string query_str = fmt::format( "select lsn, data FROM pg_logical_slot_peek_binary_changes(" "'{}', NULL, NULL, 'publication_names', '{}', 'proto_version', '1')", replication_slot_name, publication_name); pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); + LOG_DEBUG(log, "Starting replication stream"); while (true) { @@ -364,14 +404,12 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() LOG_TRACE(log, "STREAM REPLICATION END"); stream.complete(); - Block result_rows = description.sample_block.cloneWithColumns(std::move(columns)); - if (result_rows.rows()) + if (slot_empty) { - syncIntoTable(result_rows); - advanceLSN(tx); + tx->commit(); + return false; } - tx->commit(); break; } @@ -379,7 +417,7 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() current_lsn.lsn = (*row)[0]; LOG_TRACE(log, "Replication message: {}", (*row)[1]); - decodeReplicationMessage((*row)[1].c_str(), (*row)[1].size()); + processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } } catch (...) @@ -388,7 +426,21 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() return false; } - return !slot_empty; + Block result_rows = description.sample_block.cloneWithColumns(std::move(columns)); + if (result_rows.rows()) + { + assert(!slot_empty); + metadata.commitVersion([&]() + { + syncIntoTable(result_rows); + advanceLSN(tx); + + /// TODO: Can transaction still be active if got exception before commiting it? It must be closed if connection is ok. + tx->commit(); + }); + } + + return true; } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index ca357236180..b396dad80a9 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -2,6 +2,7 @@ #include "PostgreSQLConnection.h" #include +#include "PostgreSQLReplicaMetadata.h" #include #include #include @@ -47,6 +48,7 @@ public: const std::string & conn_str_, const std::string & replication_slot_name_, const std::string & publication_name_, + const std::string & metadata_path, const LSNPosition & start_lsn, const size_t max_block_size_, StoragePtr nested_storage_); @@ -61,16 +63,26 @@ private: void replicationStream(); void stopReplicationStream(); + enum class PostgreSQLQuery + { + INSERT, + UPDATE, + DELETE + }; + /// Start changes stream from WAL via copy command (up to max_block_size changes). bool readFromReplicationSlot(); - void decodeReplicationMessage(const char * replication_message, size_t size); + void processReplicationMessage(const char * replication_message, size_t size); void insertValue(std::string & value, size_t column_idx); + //static void insertValueMaterialized(IColumn & column, uint64_t value); + void insertDefaultValue(size_t column_idx); + void syncIntoTable(Block & block); void advanceLSN(std::shared_ptr ntx); /// Methods to parse replication message data. - void readTupleData(const char * message, size_t & pos, size_t size); + void readTupleData(const char * message, size_t & pos, PostgreSQLQuery type); void readString(const char * message, size_t & pos, size_t size, String & result); Int64 readInt64(const char * message, size_t & pos); Int32 readInt32(const char * message, size_t & pos); @@ -81,6 +93,7 @@ private: std::shared_ptr context; const std::string replication_slot_name; const std::string publication_name; + PostgreSQLReplicaMetadata metadata; const std::string table_name; PostgreSQLConnectionPtr connection, replication_connection; @@ -97,6 +110,8 @@ private: MutableColumns columns; /// Needed for insertPostgreSQLValue() method to parse array std::unordered_map array_info; + + size_t data_version = 1; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp new file mode 100644 index 00000000000..81b258b61c1 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -0,0 +1,75 @@ +#include "PostgreSQLReplicaMetadata.h" +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadata_file_path) + : metadata_file(metadata_file_path) + , tmp_metadata_file(metadata_file_path + ".tmp") + , data_version(1) +{ + readDataVersion(); +} + + +void PostgreSQLReplicaMetadata::readDataVersion() +{ + if (Poco::File(metadata_file).exists()) + { + LOG_INFO(&Poco::Logger::get("PostgreSQLReplicaMetadata"), + "PostgreSQLReplica metadata file exists. Starting version {}", data_version); + + ReadBufferFromFile in(metadata_file, DBMS_DEFAULT_BUFFER_SIZE); + + assertString("\nData version:\t", in); + readIntText(data_version, in); + + LOG_INFO(&Poco::Logger::get("PostgreSQLReplicaMetadata"), + "PostgreSQLReplica metadata file exists. Starting version {}", data_version); + } +} + + +void PostgreSQLReplicaMetadata::writeDataVersion() +{ + WriteBufferFromFile out(tmp_metadata_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); + writeString("\nData Version:\t" + toString(data_version), out); + + out.next(); + out.sync(); + out.close(); +} + + +/// While data is recieved, version is updated. Before table sync, write last version to tmp file. +/// Then sync data to table and rename tmp to non-tmp. +void PostgreSQLReplicaMetadata::commitVersion(const std::function & finalizeStreamFunc) +{ + writeDataVersion(); + + try + { + finalizeStreamFunc(); + Poco::File(tmp_metadata_file).renameTo(metadata_file); + } + catch (...) + { + Poco::File(tmp_metadata_file).remove(); + throw; + } +} + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h new file mode 100644 index 00000000000..87750c0e007 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h @@ -0,0 +1,31 @@ +#pragma once +#include + + +namespace DB +{ + +class PostgreSQLReplicaMetadata +{ +public: + PostgreSQLReplicaMetadata(const std::string & metadata_file_path); + + void commitVersion(const std::function & syncTableFunc); + + size_t version() + { + return data_version++; + } + +private: + void readDataVersion(); + void writeDataVersion(); + + const std::string metadata_file; + const std::string tmp_metadata_file; + + size_t data_version; + +}; + +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 7b75c42c7a8..53c3c66c504 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -2,6 +2,7 @@ #include "PostgreSQLReplicaConsumer.h" #include +#include #include #include #include @@ -38,6 +39,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , replication_slot(replication_slot_name_) , max_block_size(max_block_size_) , connection(std::make_shared(conn_str)) + , metadata_path(DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata") { /// Create a replication connection, through which it is possible to execute only commands from streaming replication protocol /// interface. Passing 'database' as the value instructs walsender to connect to the database specified in the dbname parameter, @@ -82,6 +84,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() throw; } + LOG_DEBUG(log, "PostgreSQLReplica starting replication proccess"); startReplication(); } @@ -170,17 +173,19 @@ void PostgreSQLReplicationHandler::startReplication() createReplicationSlot(ntx); } + LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Creating replication consumer"); consumer = std::make_shared( context, table_name, connection->conn_str(), replication_slot, publication_name, + metadata_path, start_lsn, max_block_size, nested_storage); - LOG_DEBUG(log, "Commiting replication transaction"); + LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Successfully created replication consumer"); ntx->commit(); consumer->startSynchronization(); @@ -303,18 +308,28 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, st void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr ntx) { + if (publication_name.empty()) + return; + std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); ntx->exec(query_str); } /// Only used when MaterializePostgreSQL table is dropped. -void PostgreSQLReplicationHandler::removeSlotAndPublication() +void PostgreSQLReplicationHandler::shutdownFinal() { + if (Poco::File(metadata_path).exists()) + Poco::File(metadata_path).remove(); + + /// TODO: another transaction might be active on this same connection. Need to make sure it does not happen. + replication_connection->conn()->close(); auto ntx = std::make_shared(*replication_connection->conn()); + dropPublication(ntx); if (isReplicationSlotExist(ntx, replication_slot)) dropReplicationSlot(ntx, replication_slot, false); + ntx->commit(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5cc4d336921..afc8a4bd213 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -3,6 +3,7 @@ #include #include "PostgreSQLConnection.h" #include "PostgreSQLReplicaConsumer.h" +#include "PostgreSQLReplicaMetadata.h" #include #include #include "pqxx/pqxx" @@ -29,7 +30,7 @@ public: void startup(StoragePtr storage_); void shutdown(); - void removeSlotAndPublication(); + void shutdownFinal(); private: using NontransactionPtr = std::shared_ptr; @@ -61,10 +62,10 @@ private: PostgreSQLConnectionPtr replication_connection; std::shared_ptr tx; + const String metadata_path; BackgroundSchedulePool::TaskHolder startup_task; std::shared_ptr consumer; StoragePtr nested_storage; - //LSNPosition start_lsn, final_lsn; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 13cd5321737..825c49668bb 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -40,7 +40,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -static auto nested_storage_suffix = "_ReplacingMergeTree"; +static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree"; + StoragePostgreSQLReplica::StoragePostgreSQLReplica( const StorageID & table_id_, @@ -57,9 +58,9 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( , replication_settings(std::move(replication_settings_)) { setInMemoryMetadata(storage_metadata); - relative_data_path.resize(relative_data_path.size() - 1); - relative_data_path += nested_storage_suffix; - + if (relative_data_path.ends_with("/")) + relative_data_path.resize(relative_data_path.size() - 1); + relative_data_path += NESTED_STORAGE_SUFFIX; replication_handler = std::make_unique( remote_database_name, @@ -132,7 +133,7 @@ ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() auto create_table_query = std::make_shared(); auto table_id = getStorageID(); - create_table_query->table = table_id.table_name + nested_storage_suffix; + create_table_query->table = table_id.table_name + NESTED_STORAGE_SUFFIX; create_table_query->database = table_id.database_name; create_table_query->if_not_exists = true; @@ -166,13 +167,63 @@ Pipe StoragePostgreSQLReplica::read( StoragePtr storage = DatabaseCatalog::instance().getTable(nested_storage->getStorageID(), *global_context); auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - const StorageMetadataPtr & nested_metadata = storage->getInMemoryMetadataPtr(); + const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); + + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); + + Block nested_header = nested_metadata->getSampleBlock(); + ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); + ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + + if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) + { + auto & tables_in_select_query = select_query->tables()->as(); + + if (!tables_in_select_query.children.empty()) + { + auto & tables_element = tables_in_select_query.children[0]->as(); + + if (tables_element.table_expression) + tables_element.table_expression->as().final = true; + } + } + + String filter_column_name; + Names require_columns_name = column_names; + ASTPtr expressions = std::make_shared(); + if (column_names_set.empty() || !column_names_set.count(sign_column.name)) + { + require_columns_name.emplace_back(sign_column.name); + + const auto & sign_column_name = std::make_shared(sign_column.name); + const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); + + expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); + filter_column_name = expressions->children.back()->getColumnName(); + + for (const auto & column_name : column_names) + expressions->children.emplace_back(std::make_shared(column_name)); + } + Pipe pipe = storage->read( - column_names, + require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); pipe.addTableLock(lock); + + if (!expressions->children.empty() && !pipe.empty()) + { + Block pipe_header = pipe.getHeader(); + auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, expression_actions, filter_column_name, false); + }); + } + return pipe; } @@ -187,7 +238,7 @@ void StoragePostgreSQLReplica::startup() if (!path.exists()) { LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), - "Creating helper table {}", table_id.table_name + nested_storage_suffix); + "Creating helper table {}", table_id.table_name + NESTED_STORAGE_SUFFIX); InterpreterCreateQuery interpreter(ast_create, context_copy); interpreter.execute(); } @@ -196,13 +247,9 @@ void StoragePostgreSQLReplica::startup() "Directory already exists {}", relative_data_path); nested_storage = DatabaseCatalog::instance().getTable( - StorageID(table_id.database_name, table_id.table_name + nested_storage_suffix), + StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), *global_context); - //nested_storage = createTableFromAST( - // ast_create->as(), getStorageID().database_name, relative_data_path, context_copy, false).second; - //nested_storage->startup(); - replication_handler->startup(nested_storage); } @@ -215,7 +262,7 @@ void StoragePostgreSQLReplica::shutdown() void StoragePostgreSQLReplica::shutdownFinal() { - replication_handler->removeSlotAndPublication(); + replication_handler->shutdownFinal(); dropNested(); } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 8dbfeb79bf0..3207389c68f 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -67,7 +67,7 @@ private: ASTPtr getCreateHelperTableQuery(); void dropNested(); - String relative_data_path; + String relative_data_path, metadata_path; std::shared_ptr global_context; std::unique_ptr replication_settings; diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp b/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp index 5d4723364dc..8cd17cca982 100644 --- a/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp @@ -16,6 +16,7 @@ #include #include + namespace DB { @@ -24,6 +25,13 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } + +void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_column) +{ + column.insertFrom(sample_column, 0); +} + + void insertPostgreSQLValue( IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.h b/src/Storages/PostgreSQL/insertPostgreSQLValue.h index 1582d35d096..d9f24247935 100644 --- a/src/Storages/PostgreSQL/insertPostgreSQLValue.h +++ b/src/Storages/PostgreSQL/insertPostgreSQLValue.h @@ -26,4 +26,6 @@ void insertPostgreSQLValue( void preparePostgreSQLArrayInfo( std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type); +void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_column); + } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 8aebae5896b..8fc7a93b579 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -17,6 +17,7 @@ namespace DB class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; +using ConnectionPtr = std::shared_ptr; class StoragePostgreSQL final : public ext::shared_ptr_helper, public IStorage { diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 487ee2a35cb..6852b8beaf1 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -31,6 +31,7 @@ def create_postgres_db(cursor, name): def create_postgres_table(cursor, table_name): cursor.execute(postgres_table_template.format(table_name)) + cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) def postgresql_replica_check_result(result, check=False, ref_file='test_postgresql_replica.reference'): fpath = p.join(p.dirname(__file__), ref_file) @@ -71,13 +72,14 @@ def test_initial_load_from_snapshot(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') - result = instance.query('SELECT * FROM test.postgresql_replica;') + time.sleep(0.2) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) @@ -90,7 +92,7 @@ def test_no_connection_at_startup(started_cluster): started_cluster.pause_container('postgres1') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -102,9 +104,8 @@ def test_no_connection_at_startup(started_cluster): while int(result) == 0: result = instance.query('SELECT count() FROM test.postgresql_replica;') time.sleep(1); - print(result) - result = instance.query('SELECT * FROM test.postgresql_replica;') + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) @@ -116,24 +117,25 @@ def test_detach_attach_is_ok(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') - result = instance.query('SELECT * FROM test.postgresql_replica;') + time.sleep(0.2) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') postgresql_replica_check_result(result, True) instance.query('DETACH TABLE test.postgresql_replica') instance.query('ATTACH TABLE test.postgresql_replica') - result = instance.query('SELECT * FROM test.postgresql_replica;') + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) -def test_replicating_inserts(started_cluster): +def test_replicating_insert_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() create_postgres_table(cursor, 'postgresql_replica'); @@ -141,19 +143,20 @@ def test_replicating_inserts(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)") instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') + time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') assert(int(result) == 10) instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)") - time.sleep(4) + time.sleep(2) result = instance.query('SELECT count() FROM test.postgresql_replica;') assert(int(result) == 30) @@ -161,13 +164,44 @@ def test_replicating_inserts(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)") - time.sleep(4) + time.sleep(2) result = instance.query('SELECT count() FROM test.postgresql_replica;') assert(int(result) == 50) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + cursor.execute('DROP TABLE postgresql_replica;') + postgresql_replica_check_result(result, True) + +def test_replicating_delete_queries(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + time.sleep(0.2) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + postgresql_replica_check_result(result, True) + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)") + time.sleep(2) + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 100) + + cursor.execute('DELETE FROM postgresql_replica WHERE key > 49;') + time.sleep(2); + + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) From 8d0c28d4706c4960dba13d056e11aecdabe3d498 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Feb 2021 09:33:31 +0000 Subject: [PATCH 008/105] Replicate update queries --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 28 +++++++++++++++++-- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 2 +- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 1 + .../test_storage_postgresql_replica/test.py | 27 ++++++++++++++++++ 4 files changed, 54 insertions(+), 4 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 3a91e893392..ae65a39ca1f 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -210,7 +210,7 @@ Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos) } -void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos, PostgreSQLQuery type) +void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos); /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data @@ -236,8 +236,6 @@ void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos { columns[num_columns]->insert(Int8(1)); columns[num_columns + 1]->insert(UInt64(metadata.version())); - //insertValueMaterialized(*columns[num_columns], 1); - //insertValueMaterialized(*columns[num_columns + 1], metadata.version()); break; } case PostgreSQLQuery::DELETE: @@ -248,6 +246,12 @@ void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos } case PostgreSQLQuery::UPDATE: { + if (old_value) + columns[num_columns]->insert(Int8(-1)); + else + columns[num_columns]->insert(Int8(1)); + + columns[num_columns + 1]->insert(UInt64(metadata.version())); break; } } @@ -319,17 +323,35 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos); Int8 new_tuple = readInt8(replication_message, pos); + LOG_DEBUG(log, "relationID {}, newTuple {}", relation_id, new_tuple); readTupleData(replication_message, pos, PostgreSQLQuery::INSERT); break; } case 'U': // Update + { + Int32 relation_id = readInt32(replication_message, pos); + Int8 primary_key_or_old_tuple_data = readInt8(replication_message, pos); + + LOG_DEBUG(log, "relationID {}, key {}", relation_id, primary_key_or_old_tuple_data); + + readTupleData(replication_message, pos, PostgreSQLQuery::UPDATE, true); + + if (pos + 1 < size) + { + Int8 new_tuple_data = readInt8(replication_message, pos); + LOG_DEBUG(log, "new tuple data {}", new_tuple_data); + readTupleData(replication_message, pos, PostgreSQLQuery::UPDATE); + } + break; + } case 'D': // Delete { Int32 relation_id = readInt32(replication_message, pos); //Int8 index_replica_identity = readInt8(replication_message, pos); Int8 full_replica_identity = readInt8(replication_message, pos); + LOG_DEBUG(log, "relationID {}, full replica identity {}", relation_id, full_replica_identity); //LOG_DEBUG(log, "relationID {}, index replica identity {} full replica identity {}", diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index b396dad80a9..0eb6be143b5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -82,7 +82,7 @@ private: void advanceLSN(std::shared_ptr ntx); /// Methods to parse replication message data. - void readTupleData(const char * message, size_t & pos, PostgreSQLQuery type); + void readTupleData(const char * message, size_t & pos, PostgreSQLQuery type, bool old_value = false); void readString(const char * message, size_t & pos, size_t size, String & result); Int64 readInt64(const char * message, size_t & pos); Int32 readInt32(const char * message, size_t & pos); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index 81b258b61c1..f3e1086bc91 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -62,6 +62,7 @@ void PostgreSQLReplicaMetadata::commitVersion(const std::function & fina try { + /// TODO: return last actially written lsn and write it to file finalizeStreamFunc(); Poco::File(tmp_metadata_file).renameTo(metadata_file); } diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 6852b8beaf1..b4549e03ced 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -206,6 +206,33 @@ def test_replicating_delete_queries(started_cluster): postgresql_replica_check_result(result, True) +def test_replicating_update_queries(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 50) + + cursor.execute('UPDATE postgresql_replica SET value = value - 10;') + time.sleep(2); + + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + print(result) + cursor.execute('DROP TABLE postgresql_replica;') + postgresql_replica_check_result(result, True) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From d255b63364232dcd3c72fd749f74d468986eb9ff Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Feb 2021 17:17:16 +0000 Subject: [PATCH 009/105] Slightly better --- .../compose/docker_compose_postgres.yml | 2 +- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 12 ++-- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 4 +- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 11 +--- .../PostgreSQL/PostgreSQLReplicaMetadata.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 56 +++++++------------ .../PostgreSQL/PostgreSQLReplicationHandler.h | 7 +-- .../test_storage_postgresql_replica/test.py | 34 ++++++++++- 8 files changed, 71 insertions(+), 57 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 7b3bee8de08..4b39623ec5e 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -7,7 +7,7 @@ services: POSTGRES_PASSWORD: mysecretpassword ports: - 5432:5432 - command: [ "postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=5"] + command: [ "postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=2"] networks: default: aliases: diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index ae65a39ca1f..b4a7344a9cd 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -43,7 +43,7 @@ static const auto max_empty_slot_reads = 16; PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( std::shared_ptr context_, const std::string & table_name_, - const std::string & conn_str, + PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, @@ -56,14 +56,12 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( , publication_name(publication_name_) , metadata(metadata_path) , table_name(table_name_) - , connection(std::make_shared(conn_str)) + , connection(std::move(connection_)) , current_lsn(start_lsn) , max_block_size(max_block_size_) , nested_storage(nested_storage_) , sample_block(nested_storage->getInMemoryMetadata().getSampleBlock()) { - replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); - description.init(sample_block); for (const auto idx : ext::range(0, description.sample_block.columns())) if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) @@ -94,6 +92,7 @@ void PostgreSQLReplicaConsumer::replicationStream() { size_t count_empty_slot_reads = 0; auto start_time = std::chrono::steady_clock::now(); + metadata.readDataVersion(); LOG_TRACE(log, "Starting replication stream"); @@ -406,9 +405,12 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() columns = description.sample_block.cloneEmptyColumns(); std::shared_ptr tx; bool slot_empty = true; + try { - tx = std::make_shared(*replication_connection->conn()); + tx = std::make_shared(*connection->conn()); + //tx->set_variable("transaction_isolation", "'repeatable read'"); + /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. std::string query_str = fmt::format( "select lsn, data FROM pg_logical_slot_peek_binary_changes(" diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 0eb6be143b5..cbe19c4436e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -45,7 +45,7 @@ public: PostgreSQLReplicaConsumer( std::shared_ptr context_, const std::string & table_name_, - const std::string & conn_str_, + PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, @@ -96,7 +96,7 @@ private: PostgreSQLReplicaMetadata metadata; const std::string table_name; - PostgreSQLConnectionPtr connection, replication_connection; + PostgreSQLConnectionPtr connection; LSNPosition current_lsn, final_lsn; BackgroundSchedulePool::TaskHolder wal_reader_task; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index f3e1086bc91..74804d0d93d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -21,7 +21,6 @@ PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadat , tmp_metadata_file(metadata_file_path + ".tmp") , data_version(1) { - readDataVersion(); } @@ -29,16 +28,12 @@ void PostgreSQLReplicaMetadata::readDataVersion() { if (Poco::File(metadata_file).exists()) { - LOG_INFO(&Poco::Logger::get("PostgreSQLReplicaMetadata"), - "PostgreSQLReplica metadata file exists. Starting version {}", data_version); - ReadBufferFromFile in(metadata_file, DBMS_DEFAULT_BUFFER_SIZE); - assertString("\nData version:\t", in); readIntText(data_version, in); - LOG_INFO(&Poco::Logger::get("PostgreSQLReplicaMetadata"), - "PostgreSQLReplica metadata file exists. Starting version {}", data_version); + LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaMetadata"), + "Last written version is {}. (From metadata file {})", data_version, metadata_file); } } @@ -46,7 +41,7 @@ void PostgreSQLReplicaMetadata::readDataVersion() void PostgreSQLReplicaMetadata::writeDataVersion() { WriteBufferFromFile out(tmp_metadata_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); - writeString("\nData Version:\t" + toString(data_version), out); + writeString("\nData version:\t" + toString(data_version), out); out.next(); out.sync(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h index 87750c0e007..13a53746c22 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h @@ -11,6 +11,7 @@ public: PostgreSQLReplicaMetadata(const std::string & metadata_file_path); void commitVersion(const std::function & syncTableFunc); + void readDataVersion(); size_t version() { @@ -18,7 +19,6 @@ public: } private: - void readDataVersion(); void writeDataVersion(); const std::string metadata_file; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 53c3c66c504..b845f697d1c 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -39,19 +39,14 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , replication_slot(replication_slot_name_) , max_block_size(max_block_size_) , connection(std::make_shared(conn_str)) + , replication_connection(std::make_shared(fmt::format("{} replication=database", connection->conn_str()))) , metadata_path(DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata") { - /// Create a replication connection, through which it is possible to execute only commands from streaming replication protocol - /// interface. Passing 'database' as the value instructs walsender to connect to the database specified in the dbname parameter, - /// which will allow the connection to be used for logical replication from that database. - replication_connection = std::make_shared(fmt::format("{} replication=database", conn_str)); - - /// Non temporary replication slot. Should be the same at restart. if (replication_slot.empty()) replication_slot = fmt::format("{}_{}_ch_replication_slot", database_name, table_name); /// Temporary replication slot is used to acquire a snapshot for initial table synchronization and to determine starting lsn position. - temp_replication_slot = replication_slot + "_temp"; + tmp_replication_slot = replication_slot + "_temp"; startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); startup_task->deactivate(); @@ -69,8 +64,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { try { - /// Used commands require a specific transaction isolation mode. - replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); + connection->conn(); } catch (pqxx::broken_connection const & pqxx_error) { @@ -133,7 +127,10 @@ void PostgreSQLReplicationHandler::createPublication() void PostgreSQLReplicationHandler::startReplication() { - tx = std::make_shared(*connection->conn()); + /// used commands require a specific transaction isolation mode. + replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); + + tx = std::make_shared(*replication_connection->conn()); if (publication_name.empty()) { publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); @@ -154,8 +151,8 @@ void PostgreSQLReplicationHandler::startReplication() auto ntx = std::make_shared(*replication_connection->conn()); /// Normally temporary replication slot should not exist. - if (isReplicationSlotExist(ntx, temp_replication_slot)) - dropReplicationSlot(ntx, temp_replication_slot, true); + if (isReplicationSlotExist(ntx, tmp_replication_slot)) + dropReplicationSlot(ntx, tmp_replication_slot); std::string snapshot_name; LSNPosition start_lsn; @@ -168,16 +165,18 @@ void PostgreSQLReplicationHandler::startReplication() /// Initial table synchronization from created snapshot loadFromSnapshot(snapshot_name); /// Do not need this replication slot anymore (snapshot loaded and start lsn determined - dropReplicationSlot(ntx, temp_replication_slot, true); + dropReplicationSlot(ntx, tmp_replication_slot); /// Non-temporary replication slot createReplicationSlot(ntx); } + ntx->commit(); + LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Creating replication consumer"); consumer = std::make_shared( context, table_name, - connection->conn_str(), + std::move(connection), replication_slot, publication_name, metadata_path, @@ -186,7 +185,6 @@ void PostgreSQLReplicationHandler::startReplication() nested_storage); LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Successfully created replication consumer"); - ntx->commit(); consumer->startSynchronization(); } @@ -254,18 +252,18 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, void PostgreSQLReplicationHandler::createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name) { - std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} TEMPORARY LOGICAL pgoutput EXPORT_SNAPSHOT", temp_replication_slot); + std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} TEMPORARY LOGICAL pgoutput EXPORT_SNAPSHOT", tmp_replication_slot); try { pqxx::result result{ntx->exec(query_str)}; start_lsn.lsn = result[0][1].as(); snapshot_name = result[0][2].as(); LOG_TRACE(log, "Created temporary replication slot: {}, start lsn: {}, snapshot: {}", - temp_replication_slot, start_lsn.lsn, snapshot_name); + tmp_replication_slot, start_lsn.lsn, snapshot_name); } catch (Exception & e) { - e.addMessage("while creating PostgreSQL replication slot {}", temp_replication_slot); + e.addMessage("while creating PostgreSQL replication slot {}", tmp_replication_slot); throw; } } @@ -287,21 +285,10 @@ void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx) } -void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, std::string & slot_name, bool use_replication_api) +void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, std::string & slot_name) { - if (use_replication_api) - { - std::string query_str = fmt::format("DROP_REPLICATION_SLOT {}", slot_name); - ntx->exec(query_str); - } - else - { - pqxx::work work(*connection->conn()); - std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); - work.exec(query_str); - work.commit(); - } - + std::string query_str = fmt::format("DROP_REPLICATION_SLOT {}", slot_name); + ntx->exec(query_str); LOG_TRACE(log, "Replication slot {} is dropped", slot_name); } @@ -319,16 +306,15 @@ void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr ntx) /// Only used when MaterializePostgreSQL table is dropped. void PostgreSQLReplicationHandler::shutdownFinal() { + /// TODO: check: if metadata file does not exist and replication slot does exist, then need to drop it at startup if (Poco::File(metadata_path).exists()) Poco::File(metadata_path).remove(); - /// TODO: another transaction might be active on this same connection. Need to make sure it does not happen. - replication_connection->conn()->close(); auto ntx = std::make_shared(*replication_connection->conn()); dropPublication(ntx); if (isReplicationSlotExist(ntx, replication_slot)) - dropReplicationSlot(ntx, replication_slot, false); + dropReplicationSlot(ntx, replication_slot); ntx->commit(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index afc8a4bd213..594f57e0dc7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -42,7 +42,7 @@ private: bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); void createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name); void createReplicationSlot(NontransactionPtr ntx); - void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name, bool use_replication_api); + void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name); void dropPublication(NontransactionPtr ntx); void startReplication(); @@ -55,11 +55,10 @@ private: const std::string database_name, table_name; std::string publication_name, replication_slot; - std::string temp_replication_slot; + std::string tmp_replication_slot; const size_t max_block_size; - PostgreSQLConnectionPtr connection; - PostgreSQLConnectionPtr replication_connection; + PostgreSQLConnectionPtr connection, replication_connection; std::shared_ptr tx; const String metadata_path; diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index b4549e03ced..44c637cc165 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -228,7 +228,39 @@ def test_replicating_update_queries(started_cluster): time.sleep(2); result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') - print(result) + cursor.execute('DROP TABLE postgresql_replica;') + postgresql_replica_check_result(result, True) + + +def test_resume_from_written_version(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)") + time.sleep(2) + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 100) + + instance.query('DETACH TABLE test.postgresql_replica') + + cursor.execute('DELETE FROM postgresql_replica WHERE key > 49;') + cursor.execute('UPDATE postgresql_replica SET value = value - 10;') + + instance.query('ATTACH TABLE test.postgresql_replica') + + time.sleep(3) + + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) From 02e19f942226184cdfa7d7827c9b8bef995253e4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Feb 2021 21:05:43 +0000 Subject: [PATCH 010/105] Better --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 30 +++---- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 6 +- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 53 +++++++++--- .../PostgreSQL/PostgreSQLReplicaMetadata.h | 12 +-- .../PostgreSQLReplicationHandler.cpp | 46 ++++++---- .../PostgreSQL/PostgreSQLReplicationHandler.h | 14 ++- .../test_storage_postgresql_replica/test.py | 85 +++++++++++++++++-- 7 files changed, 176 insertions(+), 70 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index b4a7344a9cd..e8e73cd2d52 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -67,8 +67,6 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); - columns = description.sample_block.cloneEmptyColumns(); - wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ replicationStream(); }); wal_reader_task->deactivate(); @@ -92,7 +90,7 @@ void PostgreSQLReplicaConsumer::replicationStream() { size_t count_empty_slot_reads = 0; auto start_time = std::chrono::steady_clock::now(); - metadata.readDataVersion(); + metadata.readMetadata(); LOG_TRACE(log, "Starting replication stream"); @@ -384,18 +382,19 @@ void PostgreSQLReplicaConsumer::syncIntoTable(Block & block) } -void PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr ntx) +String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr ntx) { LOG_TRACE(log, "CURRENT LSN FROM TO {}", final_lsn.lsn); - std::string query_str = fmt::format("SELECT pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn.lsn); - pqxx::result result{ntx->exec(query_str)}; - if (!result.empty()) - { - std::string s1 = result[0].size() > 0 && !result[0][0].is_null() ? result[0][0].as() : "NULL"; - std::string s2 = result[0].size() > 1 && !result[0][1].is_null() ? result[0][1].as() : "NULL"; - LOG_TRACE(log, "ADVANCE LSN: {} and {}", s1, s2); - } + std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn.lsn); + pqxx::result result{ntx->exec(query_str)}; + + ntx->commit(); + + if (!result.empty()) + return result[0][0].as(); + + return final_lsn.lsn; } @@ -454,13 +453,10 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() if (result_rows.rows()) { assert(!slot_empty); - metadata.commitVersion([&]() + metadata.commitMetadata(final_lsn.lsn, [&]() { syncIntoTable(result_rows); - advanceLSN(tx); - - /// TODO: Can transaction still be active if got exception before commiting it? It must be closed if connection is ok. - tx->commit(); + return advanceLSN(tx); }); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index cbe19c4436e..efb9dabc121 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -22,9 +22,6 @@ struct LSNPosition uint64_t upper_half, lower_half, result; std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); result = (upper_half << 32) + lower_half; - //LOG_DEBUG(&Poco::Logger::get("LSNParsing"), - // "Created replication slot. upper half: {}, lower_half: {}, start lsn: {}", - // upper_half, lower_half, result); return result; } @@ -32,7 +29,6 @@ struct LSNPosition { char result[16]; std::snprintf(result, sizeof(result), "%lX/%lX", (lsn_value >> 32), lsn_value & 0xFFFFFFFF); - //assert(lsn_value == result.getValue()); std::string ans = result; return ans; } @@ -79,7 +75,7 @@ private: void insertDefaultValue(size_t column_idx); void syncIntoTable(Block & block); - void advanceLSN(std::shared_ptr ntx); + String advanceLSN(std::shared_ptr ntx); /// Methods to parse replication message data. void readTupleData(const char * message, size_t & pos, PostgreSQLQuery type, bool old_value = false); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index 74804d0d93d..3188f271f0a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -19,29 +19,51 @@ namespace ErrorCodes PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadata_file_path) : metadata_file(metadata_file_path) , tmp_metadata_file(metadata_file_path + ".tmp") - , data_version(1) + , last_version(1) { } -void PostgreSQLReplicaMetadata::readDataVersion() +void PostgreSQLReplicaMetadata::readMetadata() { if (Poco::File(metadata_file).exists()) { ReadBufferFromFile in(metadata_file, DBMS_DEFAULT_BUFFER_SIZE); - assertString("\nData version:\t", in); - readIntText(data_version, in); + + assertString("\nLast version:\t", in); + readIntText(last_version, in); + + assertString("\nLast LSN:\t", in); + readString(last_lsn, in); + + if (checkString("\nActual LSN:\t", in)) + { + std::string actual_lsn; + readString(actual_lsn, in); + + if (!actual_lsn.empty()) + last_lsn = actual_lsn; + } LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaMetadata"), - "Last written version is {}. (From metadata file {})", data_version, metadata_file); + "Last written version is {}. (From metadata file {})", last_version, metadata_file); } } -void PostgreSQLReplicaMetadata::writeDataVersion() +void PostgreSQLReplicaMetadata::writeMetadata(bool append_metadata) { WriteBufferFromFile out(tmp_metadata_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); - writeString("\nData version:\t" + toString(data_version), out); + + if (!append_metadata) + { + writeString("\nLast version:\t" + toString(last_version), out); + writeString("\nLast LSN:\t" + toString(last_lsn), out); + } + else + { + writeString("\nActual LSN:\t" + toString(last_lsn), out); + } out.next(); out.sync(); @@ -51,14 +73,15 @@ void PostgreSQLReplicaMetadata::writeDataVersion() /// While data is recieved, version is updated. Before table sync, write last version to tmp file. /// Then sync data to table and rename tmp to non-tmp. -void PostgreSQLReplicaMetadata::commitVersion(const std::function & finalizeStreamFunc) +void PostgreSQLReplicaMetadata::commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc) { - writeDataVersion(); + std::string actual_lsn; + last_lsn = lsn; + writeMetadata(); try { - /// TODO: return last actially written lsn and write it to file - finalizeStreamFunc(); + actual_lsn = finalizeStreamFunc(); Poco::File(tmp_metadata_file).renameTo(metadata_file); } catch (...) @@ -66,6 +89,14 @@ void PostgreSQLReplicaMetadata::commitVersion(const std::function & fina Poco::File(tmp_metadata_file).remove(); throw; } + + /// This is not supposed to happen + if (actual_lsn != last_lsn) + { + writeMetadata(true); + LOG_WARNING(&Poco::Logger::get("PostgreSQLReplicaMetadata"), + "Last written LSN {} is not equal to actual LSN {}", last_lsn, actual_lsn); + } } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h index 13a53746c22..f93b74c8c65 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h @@ -10,22 +10,22 @@ class PostgreSQLReplicaMetadata public: PostgreSQLReplicaMetadata(const std::string & metadata_file_path); - void commitVersion(const std::function & syncTableFunc); - void readDataVersion(); + void commitMetadata(std::string & lsn, const std::function & syncTableFunc); + void readMetadata(); size_t version() { - return data_version++; + return last_version++; } private: - void writeDataVersion(); + void writeMetadata(bool append_metadata = false); const std::string metadata_file; const std::string tmp_metadata_file; - size_t data_version; - + uint64_t last_version; + std::string last_lsn; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index b845f697d1c..1726185ad8a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -35,6 +35,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , context(context_) , database_name(database_name_) , table_name(table_name_) + , connection_str(conn_str) , publication_name(publication_name_) , replication_slot(replication_slot_name_) , max_block_size(max_block_size_) @@ -70,6 +71,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { LOG_ERROR(log, "Unable to set up connection for table {}.{}. Reconnection attempt continues. Error message: {}", database_name, table_name, pqxx_error.what()); + startup_task->scheduleAfter(reschedule_ms); } catch (Exception & e) @@ -78,7 +80,6 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() throw; } - LOG_DEBUG(log, "PostgreSQLReplica starting replication proccess"); startReplication(); } @@ -90,7 +91,7 @@ void PostgreSQLReplicationHandler::shutdown() } -bool PostgreSQLReplicationHandler::isPublicationExist() +bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr tx) { std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); pqxx::result result{tx->exec(query_str)}; @@ -105,7 +106,7 @@ bool PostgreSQLReplicationHandler::isPublicationExist() } -void PostgreSQLReplicationHandler::createPublication() +void PostgreSQLReplicationHandler::createPublication(std::shared_ptr tx) { /// 'ONLY' means just a table, without descendants. std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, table_name); @@ -119,28 +120,29 @@ void PostgreSQLReplicationHandler::createPublication() throw Exception(fmt::format("PostgreSQL table {}.{} does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); } - /// TODO: check replica identity + /// TODO: check replica identity? /// Requires changed replica identity for included table to be able to receive old values of updated rows. - /// (ALTER TABLE table_name REPLICA IDENTITY FULL ?) } void PostgreSQLReplicationHandler::startReplication() { + LOG_DEBUG(log, "PostgreSQLReplica starting replication proccess"); + /// used commands require a specific transaction isolation mode. replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); - tx = std::make_shared(*replication_connection->conn()); + auto tx = std::make_shared(*replication_connection->conn()); if (publication_name.empty()) { publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); /// Publication defines what tables are included into replication stream. Should be deleted only if MaterializePostgreSQL /// table is dropped. - if (!isPublicationExist()) - createPublication(); + if (!isPublicationExist(tx)) + createPublication(tx); } - else if (!isPublicationExist()) + else if (!isPublicationExist(tx)) { throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -157,8 +159,7 @@ void PostgreSQLReplicationHandler::startReplication() std::string snapshot_name; LSNPosition start_lsn; - /// Non temporary replication slot should be deleted with drop table only and created only once, reused after detach. - if (!isReplicationSlotExist(ntx, replication_slot)) + auto initial_sync = [&]() { /// Temporary replication slot createTempReplicationSlot(ntx, start_lsn, snapshot_name); @@ -168,6 +169,18 @@ void PostgreSQLReplicationHandler::startReplication() dropReplicationSlot(ntx, tmp_replication_slot); /// Non-temporary replication slot createReplicationSlot(ntx); + }; + + /// Non temporary replication slot should be deleted with drop table only and created only once, reused after detach. + if (!isReplicationSlotExist(ntx, replication_slot)) + { + initial_sync(); + } + else if (!Poco::File(metadata_path).exists()) + { + /// If non-temporary slot exists and metadata file (where last synced version is written) does not exist, it is not normal. + dropReplicationSlot(ntx, replication_slot); + initial_sync(); } ntx->commit(); @@ -187,6 +200,9 @@ void PostgreSQLReplicationHandler::startReplication() LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Successfully created replication consumer"); consumer->startSynchronization(); + + /// Takes time to close + replication_connection->conn()->close(); } @@ -287,7 +303,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx) void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, std::string & slot_name) { - std::string query_str = fmt::format("DROP_REPLICATION_SLOT {}", slot_name); + std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); ntx->exec(query_str); LOG_TRACE(log, "Replication slot {} is dropped", slot_name); } @@ -303,14 +319,13 @@ void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr ntx) } -/// Only used when MaterializePostgreSQL table is dropped. void PostgreSQLReplicationHandler::shutdownFinal() { - /// TODO: check: if metadata file does not exist and replication slot does exist, then need to drop it at startup if (Poco::File(metadata_path).exists()) Poco::File(metadata_path).remove(); - auto ntx = std::make_shared(*replication_connection->conn()); + connection = std::make_shared(connection_str); + auto ntx = std::make_shared(*connection->conn()); dropPublication(ntx); if (isReplicationSlotExist(ntx, replication_slot)) @@ -319,5 +334,4 @@ void PostgreSQLReplicationHandler::shutdownFinal() ntx->commit(); } - } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 594f57e0dc7..9d2fcf9f042 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -28,16 +28,15 @@ public: const std::string & replication_slot_name_, const size_t max_block_size_); - void startup(StoragePtr storage_); + void startup(StoragePtr storage); void shutdown(); void shutdownFinal(); private: using NontransactionPtr = std::shared_ptr; - void waitConnectionAndStart(); - bool isPublicationExist(); - void createPublication(); + bool isPublicationExist(std::shared_ptr tx); + void createPublication(std::shared_ptr tx); bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); void createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name); @@ -45,22 +44,19 @@ private: void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name); void dropPublication(NontransactionPtr ntx); + void waitConnectionAndStart(); void startReplication(); void loadFromSnapshot(std::string & snapshot_name); - Context createQueryContext(); - void getTableOutput(const Context & query_context); Poco::Logger * log; std::shared_ptr context; - const std::string database_name, table_name; + const std::string database_name, table_name, connection_str; std::string publication_name, replication_slot; std::string tmp_replication_slot; const size_t max_block_size; PostgreSQLConnectionPtr connection, replication_connection; - std::shared_ptr tx; - const String metadata_path; BackgroundSchedulePool::TaskHolder startup_task; std::shared_ptr consumer; diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 44c637cc165..5f91fd2f7b4 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -78,12 +78,13 @@ def test_initial_load_from_snapshot(started_cluster): PRIMARY KEY key; ''') - time.sleep(0.2) + time.sleep(1) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) +@pytest.mark.timeout(180) def test_no_connection_at_startup(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -102,8 +103,8 @@ def test_no_connection_at_startup(started_cluster): result = instance.query('SELECT count() FROM test.postgresql_replica;') while int(result) == 0: + time.sleep(0.5); result = instance.query('SELECT count() FROM test.postgresql_replica;') - time.sleep(1); result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') @@ -123,7 +124,11 @@ def test_detach_attach_is_ok(started_cluster): PRIMARY KEY key; ''') - time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 0): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') postgresql_replica_check_result(result, True) @@ -149,7 +154,11 @@ def test_replicating_insert_queries(started_cluster): PRIMARY KEY key; ''') - time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 0): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + result = instance.query('SELECT count() FROM test.postgresql_replica;') assert(int(result) == 10) @@ -188,7 +197,11 @@ def test_replicating_delete_queries(started_cluster): PRIMARY KEY key; ''') - time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 0): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') postgresql_replica_check_result(result, True) @@ -220,8 +233,11 @@ def test_replicating_update_queries(started_cluster): PRIMARY KEY key; ''') - time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 0): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + assert(int(result) == 50) cursor.execute('UPDATE postgresql_replica SET value = value - 10;') @@ -245,6 +261,13 @@ def test_resume_from_written_version(started_cluster): PRIMARY KEY key; ''') + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 0): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + assert(int(result) == 50) + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)") time.sleep(2) @@ -265,6 +288,56 @@ def test_resume_from_written_version(started_cluster): postgresql_replica_check_result(result, True) +@pytest.mark.timeout(180) +def test_many_replication_messages(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; + ''') + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 100000): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)") + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) != 200000): + result = instance.query('SELECT count() FROM test.postgresql_replica;') + time.sleep(1) + + result = instance.query('SELECT key FROM test.postgresql_replica ORDER BY key;') + expected = instance.query("SELECT number from numbers(200000)") + assert(result == expected) + + cursor.execute('UPDATE postgresql_replica SET value = key + 1 WHERE key < 100000;') + + result = instance.query('SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;') + expected = instance.query("SELECT number from numbers(100000)") + + while (result != expected): + result = instance.query('SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;') + time.sleep(1) + + cursor.execute('DELETE FROM postgresql_replica WHERE key % 2 = 1;') + cursor.execute('DELETE FROM postgresql_replica WHERE key != value;') + + result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);') + while (int(result) != 50000): + result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);') + time.sleep(1) + + cursor.execute('DROP TABLE postgresql_replica;') + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From d87bfef890498b3d8c5c6c4770a981ef7540a990 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 6 Feb 2021 00:17:54 +0000 Subject: [PATCH 011/105] Read up to max_block_size rows --- src/Core/Settings.h | 1 - .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 11 +- .../PostgreSQLReplicationSettings.h | 1 + .../PostgreSQL/StoragePostgreSQLReplica.cpp | 5 +- .../test_storage_postgresql_replica/test.py | 131 +++++++++++++----- 5 files changed, 105 insertions(+), 44 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8c830f2dac1..9bb9ad30f15 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -60,7 +60,6 @@ class IColumn; M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \ M(Milliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.", 0) \ M(Milliseconds, rabbitmq_max_wait_ms, 5000, "The wait time for reading from RabbitMQ before retry.", 0) \ - M(UInt64, postgresql_replica_max_rows_to_insert, 65536, "Maximum number of rows in PostgreSQL batch insertion in PostgreSQLReplica storage engine", 0) \ M(UInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.", 0) \ M(UInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \ M(UInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \ diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index e8e73cd2d52..3a81c4bc887 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -408,15 +408,16 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() try { tx = std::make_shared(*connection->conn()); - //tx->set_variable("transaction_isolation", "'repeatable read'"); - /// up_to_lsn is set to NULL, up_to_n_changes is set to max_block_size. + /// Read up to max_block_size rows changes (upto_n_changes parameter). It return larger number as the limit + /// is checked only after each transaction block. + /// Returns less than max_block_changes, if reached end of wal. Sync to table in this case. std::string query_str = fmt::format( "select lsn, data FROM pg_logical_slot_peek_binary_changes(" - "'{}', NULL, NULL, 'publication_names', '{}', 'proto_version', '1')", - replication_slot_name, publication_name); + "'{}', NULL, {}, 'publication_names', '{}', 'proto_version', '1')", + replication_slot_name, max_block_size, publication_name); + pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query_str)); - LOG_DEBUG(log, "Starting replication stream"); while (true) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h index 1c3ca6ff73d..8db4c3b3bb7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h @@ -10,6 +10,7 @@ namespace DB #define LIST_OF_POSTGRESQL_REPLICATION_SETTINGS(M) \ M(String, postgresql_replication_slot_name, "", "PostgreSQL replication slot name.", 0) \ M(String, postgresql_publication_name, "", "PostgreSQL publication name.", 0) \ + M(UInt64, postgresql_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ DECLARE_SETTINGS_TRAITS(PostgreSQLReplicationSettingsTraits, LIST_OF_POSTGRESQL_REPLICATION_SETTINGS) diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 825c49668bb..9c24ffe8a43 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -69,7 +69,10 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( global_context, global_context->getMacros()->expand(replication_settings->postgresql_replication_slot_name.value), global_context->getMacros()->expand(replication_settings->postgresql_publication_name.value), - global_context->getSettingsRef().postgresql_replica_max_rows_to_insert.value + replication_settings->postgresql_max_block_size.changed + ? replication_settings->postgresql_max_block_size.value + : (global_context->getSettingsRef().max_insert_block_size.value) + ); } diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5f91fd2f7b4..8774100af1a 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -65,6 +65,7 @@ def rabbitmq_setup_teardown(): instance.query('DROP TABLE IF EXISTS test.postgresql_replica') +@pytest.mark.timeout(120) def test_initial_load_from_snapshot(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -78,13 +79,16 @@ def test_initial_load_from_snapshot(started_cluster): PRIMARY KEY key; ''') - time.sleep(1) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + while postgresql_replica_check_result(result) == False: + time.sleep(0.2) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) -@pytest.mark.timeout(180) +@pytest.mark.timeout(120) def test_no_connection_at_startup(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -111,6 +115,7 @@ def test_no_connection_at_startup(started_cluster): postgresql_replica_check_result(result, True) +@pytest.mark.timeout(120) def test_detach_attach_is_ok(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -140,6 +145,7 @@ def test_detach_attach_is_ok(started_cluster): postgresql_replica_check_result(result, True) +@pytest.mark.timeout(120) def test_replicating_insert_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -155,34 +161,32 @@ def test_replicating_insert_queries(started_cluster): ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') - while (int(result) == 0): + while (int(result) != 10): time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') - result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 10) - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)") - time.sleep(2) - result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 30) + while (int(result) != 30): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)") - time.sleep(2) - result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 50) + while (int(result) != 50): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) +@pytest.mark.timeout(120) def test_replicating_delete_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -197,28 +201,34 @@ def test_replicating_delete_queries(started_cluster): PRIMARY KEY key; ''') - result = instance.query('SELECT count() FROM test.postgresql_replica;') - while (int(result) == 0): - time.sleep(0.2) - result = instance.query('SELECT count() FROM test.postgresql_replica;') - result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + while postgresql_replica_check_result(result) == False: + time.sleep(0.2) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + postgresql_replica_check_result(result, True) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)") - time.sleep(2) result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 100) + while int(result) != 100: + time.sleep(0.5) + result = instance.query('SELECT count() FROM test.postgresql_replica;') cursor.execute('DELETE FROM postgresql_replica WHERE key > 49;') - time.sleep(2); result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + while postgresql_replica_check_result(result) == False: + time.sleep(0.5) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) +@pytest.mark.timeout(120) def test_replicating_update_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -234,20 +244,22 @@ def test_replicating_update_queries(started_cluster): ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') - while (int(result) == 0): + while (int(result) != 50): time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 50) - cursor.execute('UPDATE postgresql_replica SET value = value - 10;') - time.sleep(2); result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + while postgresql_replica_check_result(result) == False: + time.sleep(0.5) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) +@pytest.mark.timeout(120) def test_resume_from_written_version(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -262,17 +274,16 @@ def test_resume_from_written_version(started_cluster): ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') - while (int(result) == 0): + while (int(result) != 50): time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 50) - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)") - time.sleep(2) result = instance.query('SELECT count() FROM test.postgresql_replica;') - assert(int(result) == 100) + while (int(result) != 100): + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') instance.query('DETACH TABLE test.postgresql_replica') @@ -281,14 +292,16 @@ def test_resume_from_written_version(started_cluster): instance.query('ATTACH TABLE test.postgresql_replica') - time.sleep(3) - result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + while postgresql_replica_check_result(result) == False: + time.sleep(0.5) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) -@pytest.mark.timeout(180) +@pytest.mark.timeout(120) def test_many_replication_messages(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -296,10 +309,14 @@ def test_many_replication_messages(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)") instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica ( + key UInt64, value UInt64, + _sign Int8 MATERIALIZED 1, + _version UInt64 MATERIALIZED 1, + PRIMARY KEY(key)) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; + SETTINGS postgresql_max_block_size = 50000; ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') @@ -311,8 +328,9 @@ def test_many_replication_messages(started_cluster): result = instance.query('SELECT count() FROM test.postgresql_replica;') while (int(result) != 200000): - result = instance.query('SELECT count() FROM test.postgresql_replica;') time.sleep(1) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + print("INSERT OK") result = instance.query('SELECT key FROM test.postgresql_replica ORDER BY key;') expected = instance.query("SELECT number from numbers(200000)") @@ -324,20 +342,59 @@ def test_many_replication_messages(started_cluster): expected = instance.query("SELECT number from numbers(100000)") while (result != expected): - result = instance.query('SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;') time.sleep(1) + result = instance.query('SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;') + print("UPDATE OK") cursor.execute('DELETE FROM postgresql_replica WHERE key % 2 = 1;') cursor.execute('DELETE FROM postgresql_replica WHERE key != value;') result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);') while (int(result) != 50000): - result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);') time.sleep(1) + result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);') + print("DELETE OK") cursor.execute('DROP TABLE postgresql_replica;') +@pytest.mark.timeout(120) +def test_flush_by_block_size(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(1000)") + + instance.query(''' + CREATE TABLE test.postgresql_replica ( + key UInt64, value UInt64, + _sign Int8 MATERIALIZED 1, + _version UInt64 MATERIALIZED 1, + PRIMARY KEY(key)) + ENGINE = PostgreSQLReplica( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + SETTINGS postgresql_max_block_size = 5000; + ''') + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while int(result) != 1000: + time.sleep(0.2) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + for i in range(100): + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT {} * 1000 + number, number from numbers(1000)".format(i)) + + time.sleep(0.5) + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while (int(result) == 0): + result = instance.query('SELECT count() FROM test.postgresql_replica;') + time.sleep(0.2) + + assert(int(result) % 5000 == 0) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 7ceb784d1e187c173438cac5d072b5d34698f1d6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 6 Feb 2021 12:28:42 +0000 Subject: [PATCH 012/105] Better slot usage, some fixes --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 82 +++++++++---------- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 34 ++------ .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 1 + .../PostgreSQL/PostgreSQLReplicaMetadata.h | 6 +- .../PostgreSQLReplicationHandler.cpp | 68 ++++----------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 19 ++--- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 59 ++++++------- .../PostgreSQL/StoragePostgreSQLReplica.h | 13 ++- .../PostgreSQL/insertPostgreSQLValue.cpp | 4 +- .../test_storage_postgresql_replica/test.py | 44 +++++----- 10 files changed, 129 insertions(+), 201 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 3a81c4bc887..d8bee013c51 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -1,31 +1,14 @@ #include "PostgreSQLReplicaConsumer.h" -#include -#include - -#include - -#include -#include -#include - -#include +#include #include - +#include +#include +#include +#include #include #include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include namespace DB { @@ -37,7 +20,6 @@ namespace ErrorCodes static const auto reschedule_ms = 500; static const auto max_thread_work_duration_ms = 60000; -static const auto max_empty_slot_reads = 16; PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( @@ -47,7 +29,7 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, - const LSNPosition & start_lsn, + const std::string & start_lsn, const size_t max_block_size_, StoragePtr nested_storage_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) @@ -69,12 +51,21 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ replicationStream(); }); wal_reader_task->deactivate(); - } void PostgreSQLReplicaConsumer::startSynchronization() { + metadata.readMetadata(); + + if (!metadata.lsn().empty()) + { + auto tx = std::make_shared(*connection->conn()); + final_lsn = metadata.lsn(); + final_lsn = advanceLSN(tx); + tx->commit(); + } + wal_reader_task->activateAndSchedule(); } @@ -88,21 +79,14 @@ void PostgreSQLReplicaConsumer::stopSynchronization() void PostgreSQLReplicaConsumer::replicationStream() { - size_t count_empty_slot_reads = 0; auto start_time = std::chrono::steady_clock::now(); - metadata.readMetadata(); LOG_TRACE(log, "Starting replication stream"); while (!stop_synchronization) { - if (!readFromReplicationSlot() && ++count_empty_slot_reads == max_empty_slot_reads) - { - LOG_TRACE(log, "Reschedule replication stream. Replication slot is empty."); + if (!readFromReplicationSlot()) break; - } - else - count_empty_slot_reads = 0; auto end_time = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); @@ -270,7 +254,6 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati Int64 transaction_commit_timestamp = readInt64(replication_message, pos); LOG_DEBUG(log, "transaction lsn {}, transaction commit timespamp {}", transaction_end_lsn, transaction_commit_timestamp); - //current_lsn.lsn_value = transaction_end_lsn; break; } case 'C': // Commit @@ -282,7 +265,7 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati Int64 transaction_commit_timestamp = readInt64(replication_message, pos); LOG_DEBUG(log, "commit lsn {}, transaction lsn {}, transaction commit timestamp {}", commit_lsn, transaction_end_lsn, transaction_commit_timestamp); - final_lsn.lsn = current_lsn.lsn; + final_lsn = current_lsn; break; } case 'O': // Origin @@ -384,9 +367,9 @@ void PostgreSQLReplicaConsumer::syncIntoTable(Block & block) String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr ntx) { - LOG_TRACE(log, "CURRENT LSN FROM TO {}", final_lsn.lsn); + LOG_TRACE(log, "CURRENT LSN FROM TO {}", final_lsn); - std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn.lsn); + std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn); pqxx::result result{ntx->exec(query_str)}; ntx->commit(); @@ -394,7 +377,7 @@ String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr(); - return final_lsn.lsn; + return final_lsn; } @@ -409,9 +392,10 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { tx = std::make_shared(*connection->conn()); - /// Read up to max_block_size rows changes (upto_n_changes parameter). It return larger number as the limit + /// Read up to max_block_size rows changes (upto_n_changes parameter). It might return larger number as the limit /// is checked only after each transaction block. /// Returns less than max_block_changes, if reached end of wal. Sync to table in this case. + std::string query_str = fmt::format( "select lsn, data FROM pg_logical_slot_peek_binary_changes(" "'{}', NULL, {}, 'publication_names', '{}', 'proto_version', '1')", @@ -439,11 +423,26 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() slot_empty = false; - current_lsn.lsn = (*row)[0]; + current_lsn = (*row)[0]; LOG_TRACE(log, "Replication message: {}", (*row)[1]); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } } + catch (const pqxx::sql_error & e) + { + /// Currently `sql replication interface` is used and it has the problem that it registers relcache + /// callbacks on each pg_logical_slot_get_changes and there is no way to invalidate them: + /// https://github.com/postgres/postgres/blob/master/src/backend/replication/pgoutput/pgoutput.c#L1128 + /// So at some point will get out of limit and then they will be cleaned. + + std::string error_message = e.what(); + if (error_message.find("out of relcache_callback_list slots") != std::string::npos) + LOG_DEBUG(log, "Out of rel_cache_list slot"); + else + tryLogCurrentException(__PRETTY_FUNCTION__); + + return false; + } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); @@ -453,8 +452,9 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() Block result_rows = description.sample_block.cloneWithColumns(std::move(columns)); if (result_rows.rows()) { + LOG_TRACE(log, "SYNCING TABLE {} max_block_size {}", result_rows.rows(), max_block_size); assert(!slot_empty); - metadata.commitMetadata(final_lsn.lsn, [&]() + metadata.commitMetadata(final_lsn, [&]() { syncIntoTable(result_rows); return advanceLSN(tx); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index efb9dabc121..41e636705b9 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -1,40 +1,18 @@ #pragma once #include "PostgreSQLConnection.h" -#include #include "PostgreSQLReplicaMetadata.h" +#include "pqxx/pqxx" + +#include #include #include -#include -#include "pqxx/pqxx" #include + namespace DB { -struct LSNPosition -{ - std::string lsn; - int64_t lsn_value; - - int64_t getValue() - { - uint64_t upper_half, lower_half, result; - std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); - result = (upper_half << 32) + lower_half; - return result; - } - - std::string getString() - { - char result[16]; - std::snprintf(result, sizeof(result), "%lX/%lX", (lsn_value >> 32), lsn_value & 0xFFFFFFFF); - std::string ans = result; - return ans; - } -}; - - class PostgreSQLReplicaConsumer { public: @@ -45,7 +23,7 @@ public: const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, - const LSNPosition & start_lsn, + const std::string & start_lsn, const size_t max_block_size_, StoragePtr nested_storage_); @@ -94,7 +72,7 @@ private: const std::string table_name; PostgreSQLConnectionPtr connection; - LSNPosition current_lsn, final_lsn; + std::string current_lsn, final_lsn; BackgroundSchedulePool::TaskHolder wal_reader_task; //BackgroundSchedulePool::TaskHolder table_sync_task; std::atomic stop_synchronization = false; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index 3188f271f0a..a5ae25c3f53 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -1,4 +1,5 @@ #include "PostgreSQLReplicaMetadata.h" + #include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h index f93b74c8c65..31044dc3490 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h @@ -13,10 +13,8 @@ public: void commitMetadata(std::string & lsn, const std::function & syncTableFunc); void readMetadata(); - size_t version() - { - return last_version++; - } + size_t version() { return last_version++; } + std::string lsn() { return last_lsn; } private: void writeMetadata(bool append_metadata = false); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 1726185ad8a..c7429e792ae 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,15 +1,10 @@ #include "PostgreSQLReplicationHandler.h" -#include "PostgreSQLReplicaConsumer.h" -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include + namespace DB { @@ -27,6 +22,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const std::string & table_name_, const std::string & conn_str, + const std::string & metadata_path_, std::shared_ptr context_, const std::string & publication_name_, const std::string & replication_slot_name_, @@ -36,19 +32,16 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , database_name(database_name_) , table_name(table_name_) , connection_str(conn_str) + , metadata_path(metadata_path_) , publication_name(publication_name_) , replication_slot(replication_slot_name_) , max_block_size(max_block_size_) , connection(std::make_shared(conn_str)) , replication_connection(std::make_shared(fmt::format("{} replication=database", connection->conn_str()))) - , metadata_path(DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata") { if (replication_slot.empty()) replication_slot = fmt::format("{}_{}_ch_replication_slot", database_name, table_name); - /// Temporary replication slot is used to acquire a snapshot for initial table synchronization and to determine starting lsn position. - tmp_replication_slot = replication_slot + "_temp"; - startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); startup_task->deactivate(); } @@ -56,7 +49,9 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( void PostgreSQLReplicationHandler::startup(StoragePtr storage) { - nested_storage = storage; + nested_storage = std::move(storage); + + startup_task->activateAndSchedule(); } @@ -67,7 +62,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { connection->conn(); } - catch (pqxx::broken_connection const & pqxx_error) + catch (const pqxx::broken_connection & pqxx_error) { LOG_ERROR(log, "Unable to set up connection for table {}.{}. Reconnection attempt continues. Error message: {}", database_name, table_name, pqxx_error.what()); @@ -152,33 +147,22 @@ void PostgreSQLReplicationHandler::startReplication() auto ntx = std::make_shared(*replication_connection->conn()); - /// Normally temporary replication slot should not exist. - if (isReplicationSlotExist(ntx, tmp_replication_slot)) - dropReplicationSlot(ntx, tmp_replication_slot); - - std::string snapshot_name; - LSNPosition start_lsn; + std::string snapshot_name, start_lsn; auto initial_sync = [&]() { - /// Temporary replication slot - createTempReplicationSlot(ntx, start_lsn, snapshot_name); - /// Initial table synchronization from created snapshot + createReplicationSlot(ntx, start_lsn, snapshot_name); loadFromSnapshot(snapshot_name); - /// Do not need this replication slot anymore (snapshot loaded and start lsn determined - dropReplicationSlot(ntx, tmp_replication_slot); - /// Non-temporary replication slot - createReplicationSlot(ntx); }; - /// Non temporary replication slot should be deleted with drop table only and created only once, reused after detach. + /// Replication slot should be deleted with drop table only and created only once, reused after detach. if (!isReplicationSlotExist(ntx, replication_slot)) { initial_sync(); } else if (!Poco::File(metadata_path).exists()) { - /// If non-temporary slot exists and metadata file (where last synced version is written) does not exist, it is not normal. + /// If replication slot exists and metadata file (where last synced version is written) does not exist, it is not normal. dropReplicationSlot(ntx, replication_slot); initial_sync(); } @@ -266,32 +250,16 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, } -void PostgreSQLReplicationHandler::createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name) +void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name) { - std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} TEMPORARY LOGICAL pgoutput EXPORT_SNAPSHOT", tmp_replication_slot); + std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", replication_slot); try { pqxx::result result{ntx->exec(query_str)}; - start_lsn.lsn = result[0][1].as(); + start_lsn = result[0][1].as(); snapshot_name = result[0][2].as(); LOG_TRACE(log, "Created temporary replication slot: {}, start lsn: {}, snapshot: {}", - tmp_replication_slot, start_lsn.lsn, snapshot_name); - } - catch (Exception & e) - { - e.addMessage("while creating PostgreSQL replication slot {}", tmp_replication_slot); - throw; - } -} - - -void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx) -{ - std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput", replication_slot); - try - { - pqxx::result result{ntx->exec(query_str)}; - LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, result[0][1].as()); + replication_slot, start_lsn, snapshot_name); } catch (Exception & e) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 9d2fcf9f042..29e7f9b3a43 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -1,17 +1,10 @@ #pragma once -#include #include "PostgreSQLConnection.h" #include "PostgreSQLReplicaConsumer.h" #include "PostgreSQLReplicaMetadata.h" -#include -#include -#include "pqxx/pqxx" -/* Implementation of logical streaming replication protocol: https://www.postgresql.org/docs/10/protocol-logical-replication.html. - */ - namespace DB { @@ -23,6 +16,7 @@ public: const std::string & database_name_, const std::string & table_name_, const std::string & conn_str_, + const std::string & metadata_path_, std::shared_ptr context_, const std::string & publication_slot_name_, const std::string & replication_slot_name_, @@ -36,11 +30,11 @@ private: using NontransactionPtr = std::shared_ptr; bool isPublicationExist(std::shared_ptr tx); - void createPublication(std::shared_ptr tx); - bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); - void createTempReplicationSlot(NontransactionPtr ntx, LSNPosition & start_lsn, std::string & snapshot_name); - void createReplicationSlot(NontransactionPtr ntx); + + void createPublication(std::shared_ptr tx); + void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name); + void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name); void dropPublication(NontransactionPtr ntx); @@ -50,14 +44,13 @@ private: Poco::Logger * log; std::shared_ptr context; - const std::string database_name, table_name, connection_str; + const std::string database_name, table_name, connection_str, metadata_path; std::string publication_name, replication_slot; std::string tmp_replication_slot; const size_t max_block_size; PostgreSQLConnectionPtr connection, replication_connection; - const String metadata_path; BackgroundSchedulePool::TaskHolder startup_task; std::shared_ptr consumer; StoragePtr nested_storage; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 9c24ffe8a43..dae51cac76f 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -1,36 +1,25 @@ #include "StoragePostgreSQLReplica.h" - -#include -#include -#include - -#include -#include -#include - -#include -#include +#include "PostgreSQLReplicationSettings.h" #include #include - #include #include - +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include #include #include #include - #include -#include "PostgreSQLReplicationSettings.h" -#include - -#include -#include namespace DB { @@ -62,10 +51,14 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( relative_data_path.resize(relative_data_path.size() - 1); relative_data_path += NESTED_STORAGE_SUFFIX; + auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath() + + "/.metadata_" + table_id_.database_name + "_" + table_id_.table_name; + replication_handler = std::make_unique( remote_database_name, remote_table_name, connection_str, + metadata_path, global_context, global_context->getMacros()->expand(replication_settings->postgresql_replication_slot_name.value), global_context->getMacros()->expand(replication_settings->postgresql_publication_name.value), @@ -148,9 +141,6 @@ ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST(); if (primary_key_ast) storage->set(storage->order_by, primary_key_ast); - /// else - - //storage->set(storage->partition_by, ?); create_table_query->set(create_table_query->storage, storage); @@ -167,17 +157,24 @@ Pipe StoragePostgreSQLReplica::read( size_t max_block_size, unsigned num_streams) { - StoragePtr storage = DatabaseCatalog::instance().getTable(nested_storage->getStorageID(), *global_context); + if (!nested_storage) + { + auto table_id = getStorageID(); + nested_storage = DatabaseCatalog::instance().getTable( + StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), + *global_context); + } + auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); - NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - Block nested_header = nested_metadata->getSampleBlock(); ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); + if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) { auto & tables_in_select_query = select_query->tables()->as(); @@ -208,7 +205,7 @@ Pipe StoragePostgreSQLReplica::read( expressions->children.emplace_back(std::make_shared(column_name)); } - Pipe pipe = storage->read( + Pipe pipe = nested_storage->read( require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); @@ -249,11 +246,9 @@ void StoragePostgreSQLReplica::startup() LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), "Directory already exists {}", relative_data_path); - nested_storage = DatabaseCatalog::instance().getTable( - StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), - *global_context); - - replication_handler->startup(nested_storage); + replication_handler->startup( + DatabaseCatalog::instance().getTable( + StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), *global_context)); } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 3207389c68f..652f948bf79 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -2,24 +2,21 @@ #include "config_core.h" +#include "PostgreSQLReplicationHandler.h" +#include "PostgreSQLReplicationSettings.h" + #include #include #include #include #include #include - #include #include #include #include - -#include #include -#include -#include "PostgreSQLReplicationHandler.h" -#include "PostgreSQLReplicationSettings.h" -#include "pqxx/pqxx" + namespace DB { @@ -67,7 +64,7 @@ private: ASTPtr getCreateHelperTableQuery(); void dropNested(); - String relative_data_path, metadata_path; + String relative_data_path; std::shared_ptr global_context; std::unique_ptr replication_settings; diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp b/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp index 8cd17cca982..20b88bff499 100644 --- a/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp @@ -1,4 +1,5 @@ #include "insertPostgreSQLValue.h" + #include #include #include @@ -10,11 +11,8 @@ #include #include #include -#include #include #include -#include -#include namespace DB diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 8774100af1a..8773c484039 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -323,6 +323,7 @@ def test_many_replication_messages(started_cluster): while (int(result) == 100000): time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') + print("SYNC OK") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)") @@ -358,41 +359,40 @@ def test_many_replication_messages(started_cluster): cursor.execute('DROP TABLE postgresql_replica;') -@pytest.mark.timeout(120) -def test_flush_by_block_size(started_cluster): +@pytest.mark.timeout(180) +def test_connection_loss(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() create_postgres_table(cursor, 'postgresql_replica'); - - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(1000)") + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query(''' - CREATE TABLE test.postgresql_replica ( - key UInt64, value UInt64, - _sign Int8 MATERIALIZED 1, - _version UInt64 MATERIALIZED 1, - PRIMARY KEY(key)) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - SETTINGS postgresql_max_block_size = 5000; + PRIMARY KEY key; ''') - result = instance.query('SELECT count() FROM test.postgresql_replica;') - while int(result) != 1000: - time.sleep(0.2) - result = instance.query('SELECT count() FROM test.postgresql_replica;') + i = 50 + while i < 100000: + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format(i)) + i += 10000 - for i in range(100): - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT {} * 1000 + number, number from numbers(1000)".format(i)) - - time.sleep(0.5) + started_cluster.pause_container('postgres1') result = instance.query('SELECT count() FROM test.postgresql_replica;') - while (int(result) == 0): - result = instance.query('SELECT count() FROM test.postgresql_replica;') - time.sleep(0.2) + print(int(result)) + time.sleep(6) - assert(int(result) % 5000 == 0) + started_cluster.unpause_container('postgres1') + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while int(result) < 100050: + time.sleep(1) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + cursor.execute('DROP TABLE postgresql_replica;') + assert(int(result) == 100050) if __name__ == '__main__': From 9c2c7be4cf7710bb0b83dc169d7f2dba9aa16494 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Feb 2021 19:32:30 +0000 Subject: [PATCH 013/105] Separate replication interface from single storage --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 155 +++++++++++------- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 58 ++++--- ...ings.cpp => PostgreSQLReplicaSettings.cpp} | 6 +- ...Settings.h => PostgreSQLReplicaSettings.h} | 6 +- .../PostgreSQLReplicationHandler.cpp | 87 +++++----- .../PostgreSQL/PostgreSQLReplicationHandler.h | 15 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 14 +- .../PostgreSQL/StoragePostgreSQLReplica.h | 8 +- 8 files changed, 209 insertions(+), 140 deletions(-) rename src/Storages/PostgreSQL/{PostgreSQLReplicationSettings.cpp => PostgreSQLReplicaSettings.cpp} (76%) rename src/Storages/PostgreSQL/{PostgreSQLReplicationSettings.h => PostgreSQLReplicaSettings.h} (63%) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index d8bee013c51..842c5b3a5d5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; } static const auto reschedule_ms = 500; @@ -24,30 +25,27 @@ static const auto max_thread_work_duration_ms = 60000; PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( std::shared_ptr context_, - const std::string & table_name_, PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, - StoragePtr nested_storage_) + Storages storages_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) , context(context_) , replication_slot_name(replication_slot_name_) , publication_name(publication_name_) , metadata(metadata_path) - , table_name(table_name_) , connection(std::move(connection_)) , current_lsn(start_lsn) , max_block_size(max_block_size_) - , nested_storage(nested_storage_) - , sample_block(nested_storage->getInMemoryMetadata().getSampleBlock()) + , storages(storages_) { - description.init(sample_block); - for (const auto idx : ext::range(0, description.sample_block.columns())) - if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) - preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); + for (const auto & [table_name, storage] : storages) + { + buffers.emplace(table_name, BufferData(storage->getInMemoryMetadata().getSampleBlock())); + } wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ replicationStream(); }); wal_reader_task->deactivate(); @@ -102,35 +100,37 @@ void PostgreSQLReplicaConsumer::replicationStream() } -void PostgreSQLReplicaConsumer::insertValue(std::string & value, size_t column_idx) +void PostgreSQLReplicaConsumer::insertValue(BufferData & buffer, const std::string & value, size_t column_idx) { LOG_TRACE(log, "INSERTING VALUE {}", value); - const auto & sample = description.sample_block.getByPosition(column_idx); - bool is_nullable = description.types[column_idx].second; + const auto & sample = buffer.description.sample_block.getByPosition(column_idx); + bool is_nullable = buffer.description.types[column_idx].second; if (is_nullable) { - ColumnNullable & column_nullable = assert_cast(*columns[column_idx]); + ColumnNullable & column_nullable = assert_cast(*buffer.columns[column_idx]); const auto & data_type = assert_cast(*sample.type); insertPostgreSQLValue( column_nullable.getNestedColumn(), value, - description.types[column_idx].first, data_type.getNestedType(), array_info, column_idx); + buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx); column_nullable.getNullMapData().emplace_back(0); } else { insertPostgreSQLValue( - *columns[column_idx], value, description.types[column_idx].first, sample.type, array_info, column_idx); + *buffer.columns[column_idx], value, + buffer.description.types[column_idx].first, sample.type, + buffer.array_info, column_idx); } } -void PostgreSQLReplicaConsumer::insertDefaultValue(size_t column_idx) +void PostgreSQLReplicaConsumer::insertDefaultValue(BufferData & buffer, size_t column_idx) { - const auto & sample = description.sample_block.getByPosition(column_idx); - insertDefaultPostgreSQLValue(*columns[column_idx], *sample.column); + const auto & sample = buffer.description.sample_block.getByPosition(column_idx); + insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column); } @@ -191,7 +191,8 @@ Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos) } -void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos, PostgreSQLQuery type, bool old_value) +void PostgreSQLReplicaConsumer::readTupleData( + BufferData & buffer, const char * message, size_t & pos, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos); /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data @@ -206,7 +207,7 @@ void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos value += readInt8(message, pos); } - insertValue(value, column_idx); + insertValue(buffer, value, column_idx); LOG_DEBUG(log, "identifier {}, col_len {}, value {}", identifier, col_len, value); } @@ -215,31 +216,35 @@ void PostgreSQLReplicaConsumer::readTupleData(const char * message, size_t & pos { case PostgreSQLQuery::INSERT: { - columns[num_columns]->insert(Int8(1)); - columns[num_columns + 1]->insert(UInt64(metadata.version())); + buffer.columns[num_columns]->insert(Int8(1)); + buffer.columns[num_columns + 1]->insert(UInt64(metadata.version())); + break; } case PostgreSQLQuery::DELETE: { - columns[num_columns]->insert(Int8(-1)); - columns[num_columns + 1]->insert(UInt64(metadata.version())); + buffer.columns[num_columns]->insert(Int8(-1)); + buffer.columns[num_columns + 1]->insert(UInt64(metadata.version())); + break; } case PostgreSQLQuery::UPDATE: { if (old_value) - columns[num_columns]->insert(Int8(-1)); + buffer.columns[num_columns]->insert(Int8(-1)); else - columns[num_columns]->insert(Int8(1)); + buffer.columns[num_columns]->insert(Int8(1)); + + buffer.columns[num_columns + 1]->insert(UInt64(metadata.version())); - columns[num_columns + 1]->insert(UInt64(metadata.version())); break; } } } -void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replication_message, size_t size) +void PostgreSQLReplicaConsumer::processReplicationMessage( + const char * replication_message, size_t size, std::unordered_set & tables_to_sync) { /// Skip '\x' size_t pos = 2; @@ -295,6 +300,17 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati LOG_DEBUG(log, "Key {}, column name {}, data type id {}, type modifier {}", key, column_name, data_type_id, type_modifier); } + table_to_insert = relation_name; + if (storages.find(table_to_insert) == storages.end()) + { + throw Exception(ErrorCodes::UNKNOWN_TABLE, + "Table {} does not exist, but is included in replication stream", table_to_insert); + } + [[maybe_unused]] auto buffer_iter = buffers.find(table_to_insert); + assert(buffer_iter != buffers.end()); + + tables_to_sync.insert(relation_name); + break; } case 'Y': // Type @@ -305,7 +321,13 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati Int8 new_tuple = readInt8(replication_message, pos); LOG_DEBUG(log, "relationID {}, newTuple {}", relation_id, new_tuple); - readTupleData(replication_message, pos, PostgreSQLQuery::INSERT); + auto buffer = buffers.find(table_to_insert); + if (buffer == buffers.end()) + { + throw Exception(ErrorCodes::UNKNOWN_TABLE, + "Buffer for table {} does not exist", table_to_insert); + } + readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::INSERT); break; } case 'U': // Update @@ -315,13 +337,14 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati LOG_DEBUG(log, "relationID {}, key {}", relation_id, primary_key_or_old_tuple_data); - readTupleData(replication_message, pos, PostgreSQLQuery::UPDATE, true); + auto buffer = buffers.find(table_to_insert); + readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::UPDATE, true); if (pos + 1 < size) { Int8 new_tuple_data = readInt8(replication_message, pos); LOG_DEBUG(log, "new tuple data {}", new_tuple_data); - readTupleData(replication_message, pos, PostgreSQLQuery::UPDATE); + readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::UPDATE); } break; @@ -334,9 +357,9 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati LOG_DEBUG(log, "relationID {}, full replica identity {}", relation_id, full_replica_identity); - //LOG_DEBUG(log, "relationID {}, index replica identity {} full replica identity {}", - // relation_id, index_replica_identity, full_replica_identity); - readTupleData(replication_message, pos, PostgreSQLQuery::DELETE); + + auto buffer = buffers.find(table_to_insert); + readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::DELETE); break; } case 'T': // Truncate @@ -348,20 +371,43 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati } -void PostgreSQLReplicaConsumer::syncIntoTable(Block & block) +void PostgreSQLReplicaConsumer::syncTables( + std::shared_ptr tx, const std::unordered_set & tables_to_sync) { - Context insert_context(*context); - insert_context.makeQueryContext(); + for (const auto & table_name : tables_to_sync) + { + auto & buffer = buffers.find(table_name)->second; + Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns)); - auto insert = std::make_shared(); - insert->table_id = nested_storage->getStorageID(); + if (result_rows.rows()) + { + LOG_TRACE(log, "SYNCING TABLE {} max_block_size {}", result_rows.rows(), max_block_size); - InterpreterInsertQuery interpreter(insert, insert_context); - auto block_io = interpreter.execute(); - OneBlockInputStream input(block); + metadata.commitMetadata(final_lsn, [&]() + { + Context insert_context(*context); + insert_context.makeQueryContext(); - copyData(input, *block_io.out); - LOG_TRACE(log, "TABLE SYNC END"); + auto insert = std::make_shared(); + insert->table_id = storages[table_name]->getStorageID(); + + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); + + /// TODO: what if one block is not enough + OneBlockInputStream input(result_rows); + + copyData(input, *block_io.out); + + LOG_TRACE(log, "TABLE SYNC END"); + + auto actual_lsn = advanceLSN(tx); + buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); + + return actual_lsn; + }); + } + } } @@ -384,9 +430,9 @@ String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr tx; bool slot_empty = true; + std::unordered_set tables_to_sync; try { @@ -425,7 +471,7 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() current_lsn = (*row)[0]; LOG_TRACE(log, "Replication message: {}", (*row)[1]); - processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); + processReplicationMessage((*row)[1].c_str(), (*row)[1].size(), tables_to_sync); } } catch (const pqxx::sql_error & e) @@ -443,23 +489,16 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() return false; } - catch (...) + catch (const Exception & e) { + if (e.code() == ErrorCodes::UNKNOWN_TABLE) + throw; + tryLogCurrentException(__PRETTY_FUNCTION__); return false; } - Block result_rows = description.sample_block.cloneWithColumns(std::move(columns)); - if (result_rows.rows()) - { - LOG_TRACE(log, "SYNCING TABLE {} max_block_size {}", result_rows.rows(), max_block_size); - assert(!slot_empty); - metadata.commitMetadata(final_lsn, [&]() - { - syncIntoTable(result_rows); - return advanceLSN(tx); - }); - } + syncTables(tx, tables_to_sync); return true; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 41e636705b9..817c57a99fa 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -16,16 +17,17 @@ namespace DB class PostgreSQLReplicaConsumer { public: + using Storages = std::unordered_map; + PostgreSQLReplicaConsumer( std::shared_ptr context_, - const std::string & table_name_, PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, - StoragePtr nested_storage_); + Storages storages_); /// Start reading WAL from current_lsn position. Initial data sync from created snapshot already done. void startSynchronization(); @@ -44,19 +46,37 @@ private: DELETE }; - /// Start changes stream from WAL via copy command (up to max_block_size changes). bool readFromReplicationSlot(); - void processReplicationMessage(const char * replication_message, size_t size); - - void insertValue(std::string & value, size_t column_idx); - //static void insertValueMaterialized(IColumn & column, uint64_t value); - void insertDefaultValue(size_t column_idx); - - void syncIntoTable(Block & block); + void syncTables(std::shared_ptr tx, const std::unordered_set & tables_to_sync); String advanceLSN(std::shared_ptr ntx); + void processReplicationMessage( + const char * replication_message, size_t size, std::unordered_set & tables_to_sync); + + struct BufferData + { + ExternalResultDescription description; + MutableColumns columns; + /// Needed for insertPostgreSQLValue() method to parse array + std::unordered_map array_info; + + BufferData(const Block block) + { + description.init(block); + columns = description.sample_block.cloneEmptyColumns(); + for (const auto idx : ext::range(0, description.sample_block.columns())) + if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) + preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); + } + }; + + using Buffers = std::unordered_map; + + void insertDefaultValue(BufferData & buffer, size_t column_idx); + void insertValue(BufferData & buffer, const std::string & value, size_t column_idx); + void readTupleData(BufferData & buffer, const char * message, size_t & pos, PostgreSQLQuery type, bool old_value = false); + /// Methods to parse replication message data. - void readTupleData(const char * message, size_t & pos, PostgreSQLQuery type, bool old_value = false); void readString(const char * message, size_t & pos, size_t size, String & result); Int64 readInt64(const char * message, size_t & pos); Int32 readInt32(const char * message, size_t & pos); @@ -69,23 +89,17 @@ private: const std::string publication_name; PostgreSQLReplicaMetadata metadata; - const std::string table_name; PostgreSQLConnectionPtr connection; std::string current_lsn, final_lsn; + const size_t max_block_size; + std::string table_to_insert; + BackgroundSchedulePool::TaskHolder wal_reader_task; - //BackgroundSchedulePool::TaskHolder table_sync_task; std::atomic stop_synchronization = false; - const size_t max_block_size; - StoragePtr nested_storage; - Block sample_block; - ExternalResultDescription description; - MutableColumns columns; - /// Needed for insertPostgreSQLValue() method to parse array - std::unordered_map array_info; - - size_t data_version = 1; + Storages storages; + Buffers buffers; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp similarity index 76% rename from src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp rename to src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp index fa5ebb0edf3..aa1fec92ef4 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp @@ -1,4 +1,4 @@ -#include "PostgreSQLReplicationSettings.h" +#include "PostgreSQLReplicaSettings.h" #include #include #include @@ -13,9 +13,9 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(PostgreSQLReplicationSettingsTraits, LIST_OF_POSTGRESQL_REPLICATION_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(PostgreSQLReplicaSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) -void PostgreSQLReplicationSettings::loadFromQuery(ASTStorage & storage_def) +void PostgreSQLReplicaSettings::loadFromQuery(ASTStorage & storage_def) { if (storage_def.settings) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h similarity index 63% rename from src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h rename to src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h index 8db4c3b3bb7..72b7f98ea6e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationSettings.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h @@ -7,14 +7,14 @@ namespace DB class ASTStorage; -#define LIST_OF_POSTGRESQL_REPLICATION_SETTINGS(M) \ +#define LIST_OF_POSTGRESQL_REPLICA_SETTINGS(M) \ M(String, postgresql_replication_slot_name, "", "PostgreSQL replication slot name.", 0) \ M(String, postgresql_publication_name, "", "PostgreSQL publication name.", 0) \ M(UInt64, postgresql_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ -DECLARE_SETTINGS_TRAITS(PostgreSQLReplicationSettingsTraits, LIST_OF_POSTGRESQL_REPLICATION_SETTINGS) +DECLARE_SETTINGS_TRAITS(PostgreSQLReplicaSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) -struct PostgreSQLReplicationSettings : public BaseSettings +struct PostgreSQLReplicaSettings : public BaseSettings { void loadFromQuery(ASTStorage & storage_def); }; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index c7429e792ae..e9d7b1d9a69 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -20,7 +20,6 @@ static const auto reschedule_ms = 500; PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, - const std::string & table_name_, const std::string & conn_str, const std::string & metadata_path_, std::shared_ptr context_, @@ -30,7 +29,6 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) , context(context_) , database_name(database_name_) - , table_name(table_name_) , connection_str(conn_str) , metadata_path(metadata_path_) , publication_name(publication_name_) @@ -40,18 +38,21 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , replication_connection(std::make_shared(fmt::format("{} replication=database", connection->conn_str()))) { if (replication_slot.empty()) - replication_slot = fmt::format("{}_{}_ch_replication_slot", database_name, table_name); + replication_slot = fmt::format("{}_ch_replication_slot", database_name); startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); startup_task->deactivate(); } -void PostgreSQLReplicationHandler::startup(StoragePtr storage) +void PostgreSQLReplicationHandler::addStoragePtr(const std::string & table_name, StoragePtr storage) { - nested_storage = std::move(storage); + storages[table_name] = std::move(storage); +} +void PostgreSQLReplicationHandler::startup() +{ startup_task->activateAndSchedule(); } @@ -64,14 +65,15 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() } catch (const pqxx::broken_connection & pqxx_error) { - LOG_ERROR(log, "Unable to set up connection for table {}.{}. Reconnection attempt continues. Error message: {}", - database_name, table_name, pqxx_error.what()); + LOG_ERROR(log, + "Unable to set up connection. Reconnection attempt continue. Error message: {}", + pqxx_error.what()); startup_task->scheduleAfter(reschedule_ms); } catch (Exception & e) { - e.addMessage("while setting up connection for {}.{}", database_name, table_name); + e.addMessage("while setting up connection for PostgreSQLReplica engine"); throw; } @@ -103,16 +105,25 @@ bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr tx) { + String table_names; + for (const auto & storage_data : storages) + { + if (!table_names.empty()) + table_names += ", "; + table_names += storage_data.first; + } + /// 'ONLY' means just a table, without descendants. - std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, table_name); + std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, table_names); try { tx->exec(query_str); LOG_TRACE(log, "Created publication {}", publication_name); } - catch (pqxx::undefined_table const &) + catch (Exception & e) { - throw Exception(fmt::format("PostgreSQL table {}.{} does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); + e.addMessage("while creating pg_publication"); + throw; } /// TODO: check replica identity? @@ -130,7 +141,7 @@ void PostgreSQLReplicationHandler::startReplication() auto tx = std::make_shared(*replication_connection->conn()); if (publication_name.empty()) { - publication_name = fmt::format("{}_{}_ch_publication", database_name, table_name); + publication_name = fmt::format("{}_ch_publication", database_name); /// Publication defines what tables are included into replication stream. Should be deleted only if MaterializePostgreSQL /// table is dropped. @@ -172,14 +183,13 @@ void PostgreSQLReplicationHandler::startReplication() LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Creating replication consumer"); consumer = std::make_shared( context, - table_name, std::move(connection), replication_slot, publication_name, metadata_path, start_lsn, max_block_size, - nested_storage); + storages); LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Successfully created replication consumer"); @@ -194,39 +204,42 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) { LOG_DEBUG(log, "Creating transaction snapshot"); - try + for (const auto & [table_name, storage] : storages) { - auto stx = std::make_unique(*connection->conn()); + try + { + auto stx = std::make_unique(*connection->conn()); - /// Specific isolation level is required to read from snapshot. - stx->set_variable("transaction_isolation", "'repeatable read'"); + /// Specific isolation level is required to read from snapshot. + stx->set_variable("transaction_isolation", "'repeatable read'"); - std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); - stx->exec(query_str); + std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); + stx->exec(query_str); - /// Load from snapshot, which will show table state before creation of replication slot. - query_str = fmt::format("SELECT * FROM {}", table_name); + /// Load from snapshot, which will show table state before creation of replication slot. + query_str = fmt::format("SELECT * FROM {}", table_name); - Context insert_context(*context); - insert_context.makeQueryContext(); + Context insert_context(*context); + insert_context.makeQueryContext(); - auto insert = std::make_shared(); - insert->table_id = nested_storage->getStorageID(); + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); - InterpreterInsertQuery interpreter(insert, insert_context); - auto block_io = interpreter.execute(); + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); - const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); - auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); + const StorageInMemoryMetadata & storage_metadata = storage->getInMemoryMetadata(); + auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - PostgreSQLBlockInputStream input(std::move(stx), query_str, sample_block, DEFAULT_BLOCK_SIZE); + PostgreSQLBlockInputStream input(std::move(stx), query_str, sample_block, DEFAULT_BLOCK_SIZE); - copyData(input, *block_io.out); - } - catch (Exception & e) - { - e.addMessage("while initial data sync for table {}.{}", database_name, table_name); - throw; + copyData(input, *block_io.out); + } + catch (Exception & e) + { + e.addMessage("while initial data synchronization"); + throw; + } } LOG_DEBUG(log, "Done loading from snapshot"); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 29e7f9b3a43..f4118c4aed8 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -14,7 +14,6 @@ public: friend class PGReplicaLSN; PostgreSQLReplicationHandler( const std::string & database_name_, - const std::string & table_name_, const std::string & conn_str_, const std::string & metadata_path_, std::shared_ptr context_, @@ -22,12 +21,14 @@ public: const std::string & replication_slot_name_, const size_t max_block_size_); - void startup(StoragePtr storage); + void startup(); + void addStoragePtr(const std::string & table_name, StoragePtr storage); void shutdown(); void shutdownFinal(); private: using NontransactionPtr = std::shared_ptr; + using Storages = std::unordered_map; bool isPublicationExist(std::shared_ptr tx); bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); @@ -44,16 +45,16 @@ private: Poco::Logger * log; std::shared_ptr context; - const std::string database_name, table_name, connection_str, metadata_path; - + const std::string database_name, connection_str, metadata_path; std::string publication_name, replication_slot; - std::string tmp_replication_slot; const size_t max_block_size; PostgreSQLConnectionPtr connection, replication_connection; - BackgroundSchedulePool::TaskHolder startup_task; std::shared_ptr consumer; - StoragePtr nested_storage; + + BackgroundSchedulePool::TaskHolder startup_task; + + Storages storages; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index dae51cac76f..27ab6c7adaf 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -1,5 +1,4 @@ #include "StoragePostgreSQLReplica.h" -#include "PostgreSQLReplicationSettings.h" #include #include @@ -35,13 +34,14 @@ static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree"; StoragePostgreSQLReplica::StoragePostgreSQLReplica( const StorageID & table_id_, const String & remote_database_name, - const String & remote_table_name, + const String & remote_table_name_, const String & connection_str, const String & relative_data_path_, const StorageInMemoryMetadata & storage_metadata, const Context & context_, - std::unique_ptr replication_settings_) + std::unique_ptr replication_settings_) : IStorage(table_id_) + , remote_table_name(remote_table_name_) , relative_data_path(relative_data_path_) , global_context(std::make_shared(context_.getGlobalContext())) , replication_settings(std::move(replication_settings_)) @@ -56,7 +56,6 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( replication_handler = std::make_unique( remote_database_name, - remote_table_name, connection_str, metadata_path, global_context, @@ -246,9 +245,12 @@ void StoragePostgreSQLReplica::startup() LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), "Directory already exists {}", relative_data_path); - replication_handler->startup( + replication_handler->addStoragePtr( + remote_table_name, DatabaseCatalog::instance().getTable( StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), *global_context)); + + replication_handler->startup(); } @@ -295,7 +297,7 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) { ASTs & engine_args = args.engine_args; bool has_settings = args.storage_def->settings; - auto postgresql_replication_settings = std::make_unique(); + auto postgresql_replication_settings = std::make_unique(); if (has_settings) postgresql_replication_settings->loadFromQuery(*args.storage_def); diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 652f948bf79..c1e4b319187 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -3,7 +3,7 @@ #include "config_core.h" #include "PostgreSQLReplicationHandler.h" -#include "PostgreSQLReplicationSettings.h" +#include "PostgreSQLReplicaSettings.h" #include #include @@ -54,7 +54,7 @@ protected: const String & relative_data_path_, const StorageInMemoryMetadata & storage_metadata, const Context & context_, - std::unique_ptr replication_settings_); + std::unique_ptr replication_settings_); private: std::shared_ptr getMaterializedColumnsDeclaration( @@ -64,10 +64,10 @@ private: ASTPtr getCreateHelperTableQuery(); void dropNested(); - String relative_data_path; + std::string remote_table_name, relative_data_path; std::shared_ptr global_context; - std::unique_ptr replication_settings; + std::unique_ptr replication_settings; std::unique_ptr replication_handler; /// ReplacingMergeTree table From ed6d19b8a6f4cfa9334b2601c5af58944b65f766 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Feb 2021 23:23:51 +0000 Subject: [PATCH 014/105] Add PostgreSQLReplica database engine --- src/Databases/DatabaseFactory.cpp | 52 +++- .../PostgreSQL/DatabasePostgreSQL.cpp | 20 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 1 - .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 253 +++++++++++++++++ .../PostgreSQL/DatabasePostgreSQLReplica.h | 88 ++++++ .../fetchPostgreSQLTableStructure.cpp | 14 + .../fetchPostgreSQLTableStructure.h | 4 +- .../PostgreSQL/PostgreSQLConnection.h | 3 +- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 188 +++++++------ .../PostgreSQL/PostgreSQLReplicaConsumer.h | 47 ++-- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 1 + .../PostgreSQL/PostgreSQLReplicaMetadata.h | 2 + .../PostgreSQLReplicationHandler.cpp | 110 ++++++-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 20 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 262 ++++++++++++------ .../PostgreSQL/StoragePostgreSQLReplica.h | 46 ++- .../__init__.py | 0 .../configs/log_conf.xml | 11 + .../test.py | 138 +++++++++ 19 files changed, 1003 insertions(+), 257 deletions(-) create mode 100644 src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp create mode 100644 src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h create mode 100644 tests/integration/test_postgresql_replica_database_engine/__init__.py create mode 100644 tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml create mode 100644 tests/integration/test_postgresql_replica_database_engine/test.py diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 5166e15b7b4..d4b7674f73b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -34,7 +34,9 @@ #if USE_LIBPQXX #include // Y_IGNORE +#include #include +#include #endif namespace DB @@ -96,7 +98,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const String & engine_name = engine_define->engine->name; const UUID & uuid = create.uuid; - if (engine_name != "MySQL" && engine_name != "MaterializeMySQL" && engine_name != "Lazy" && engine_name != "PostgreSQL" && engine_define->engine->arguments) + if (engine_name != "MySQL" && engine_name != "MaterializeMySQL" + && engine_name != "PostgreSQL" && engine_name != "PostgreSQLReplica" + && engine_name != "Lazy" && engine_define->engine->arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || @@ -219,6 +223,52 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared( context, metadata_path, engine_define, database_name, postgres_database_name, connection, use_table_cache); } + else if (engine_name == "PostgreSQLReplica") + { + const ASTFunction * engine = engine_define->engine; + + if (!engine->arguments || engine->arguments->children.size() != 4) + { + throw Exception( + fmt::format("{} Database require host:port, database_name, username, password arguments ", engine_name), + ErrorCodes::BAD_ARGUMENTS); + } + + ASTs & engine_args = engine->arguments->children; + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + const auto & host_port = safeGetLiteralValue(engine_args[0], engine_name); + const auto & postgres_database_name = safeGetLiteralValue(engine_args[1], engine_name); + const auto & username = safeGetLiteralValue(engine_args[2], engine_name); + const auto & password = safeGetLiteralValue(engine_args[3], engine_name); + + auto parsed_host_port = parseAddress(host_port, 5432); + auto connection = std::make_shared( + postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password); + + auto postgresql_replica_settings = std::make_unique(); + + if (engine_define->settings) + postgresql_replica_settings->loadFromQuery(*engine_define); + + if (create.uuid == UUIDHelpers::Nil) + { + return std::make_shared>( + context, metadata_path, uuid, engine_define, + database_name, postgres_database_name, connection, + std::move(postgresql_replica_settings)); + } + else + { + return std::make_shared>( + context, metadata_path, uuid, engine_define, + database_name, postgres_database_name, connection, + std::move(postgresql_replica_settings)); + } + } + #endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 722b9c64edb..ebe5ba107bd 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -58,7 +58,7 @@ bool DatabasePostgreSQL::empty() const { std::lock_guard lock(mutex); - auto tables_list = fetchTablesList(); + auto tables_list = fetchPostgreSQLTablesList(connection->conn()); for (const auto & table_name : tables_list) if (!detached_or_dropped.count(table_name)) @@ -74,7 +74,7 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( std::lock_guard lock(mutex); Tables tables; - auto table_names = fetchTablesList(); + auto table_names = fetchPostgreSQLTablesList(connection->conn()); for (const auto & table_name : table_names) if (!detached_or_dropped.count(table_name)) @@ -84,20 +84,6 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( } -std::unordered_set DatabasePostgreSQL::fetchTablesList() const -{ - std::unordered_set tables; - std::string query = "SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'"; - pqxx::read_transaction tx(*connection->conn()); - - for (auto table_name : tx.stream(query)) - tables.insert(std::get<0>(table_name)); - - return tables; -} - - bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const { if (table_name.find('\'') != std::string::npos @@ -299,7 +285,7 @@ void DatabasePostgreSQL::loadStoredObjects(Context & /* context */, bool, bool / void DatabasePostgreSQL::removeOutdatedTables() { std::lock_guard lock{mutex}; - auto actual_tables = fetchTablesList(); + auto actual_tables = fetchPostgreSQLTablesList(connection->conn()); if (cache_tables) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 56ea6645f15..79dbd993a0b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -80,7 +80,6 @@ private: BackgroundSchedulePool::TaskHolder cleaner_task; bool checkPostgresTable(const String & table_name) const; - std::unordered_set fetchTablesList() const; StoragePtr fetchTable(const String & table_name, const Context & context, const bool table_checked) const; void removeOutdatedTables(); ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp new file mode 100644 index 00000000000..d279d7e5c5c --- /dev/null +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -0,0 +1,253 @@ +#include + +#if USE_LIBPQXX + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; + extern const int UNKNOWN_TABLE; + extern const int TABLE_IS_DROPPED; + extern const int TABLE_ALREADY_EXISTS; +} + + +static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; + +/// TODO: add detach, after which table structure is updated, need to update StoragePtr and recreate nested_storage. +/// Also pass new storagePtr to replication Handler. Stop replication stream mean while? + +template<> +DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( + const Context & context, + const String & metadata_path_, + UUID /* uuid */, + const ASTStorage * database_engine_define_, + const String & database_name_, + const String & postgres_database_name, + PostgreSQLConnectionPtr connection_, + std::unique_ptr settings_) + : DatabaseOrdinary( + database_name_, metadata_path_, "data/" + escapeForFileName(database_name_) + "/", + "DatabasePostgreSQLReplica (" + database_name_ + ")", context) + , global_context(context.getGlobalContext()) + , metadata_path(metadata_path_) + , database_engine_define(database_engine_define_->clone()) + , database_name(database_name_) + , remote_database_name(postgres_database_name) + , connection(std::move(connection_)) + , settings(std::move(settings_)) +{ +} + + +template<> +DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( + const Context & context, + const String & metadata_path_, + UUID uuid, + const ASTStorage * database_engine_define_, + const String & database_name_, + const String & postgres_database_name, + PostgreSQLConnectionPtr connection_, + std::unique_ptr settings_) + : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabasePostgreSQLReplica (" + database_name_ + ")", context) + , global_context(context.getGlobalContext()) + , metadata_path(metadata_path_) + , database_engine_define(database_engine_define_->clone()) + , remote_database_name(postgres_database_name) + , connection(std::move(connection_)) + , settings(std::move(settings_)) +{ +} + + +template +void DatabasePostgreSQLReplica::startSynchronization() +{ + auto publication_name = global_context.getMacros()->expand(settings->postgresql_publication_name.value); + auto replication_slot = global_context.getMacros()->expand(settings->postgresql_replication_slot_name.value); + + replication_handler = std::make_unique( + remote_database_name, + connection->conn_str(), + metadata_path + METADATA_SUFFIX, + std::make_shared(global_context), + replication_slot, + publication_name, + settings->postgresql_max_block_size.changed + ? settings->postgresql_max_block_size.value + : (global_context.getSettingsRef().max_insert_block_size.value)); + + std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->conn()); + + for (const auto & table_name : tables_to_replicate) + { + auto storage = getStorage(table_name); + + if (storage) + { + replication_handler->addStorage(table_name, storage.get()->template as()); + tables[table_name] = storage; + } + } + + LOG_TRACE(&Poco::Logger::get("PostgreSQLReplicaDatabaseEngine"), "Loaded {} tables. Starting synchronization", tables.size()); + replication_handler->startup(); +} + + +template +StoragePtr DatabasePostgreSQLReplica::getStorage(const String & name) +{ + auto storage = tryGetTable(name, global_context); + + if (storage) + return storage; + + auto use_nulls = global_context.getSettingsRef().external_table_functions_use_nulls; + auto columns = fetchPostgreSQLTableStructure(connection->conn(), name, use_nulls); + + if (!columns) + return StoragePtr{}; + + StorageInMemoryMetadata metadata; + metadata.setColumns(ColumnsDescription(*columns)); + + storage = StoragePostgreSQLReplica::create(StorageID(database_name, name), metadata_path, metadata, global_context); + + return storage; +} + + +template +void DatabasePostgreSQLReplica::shutdown() +{ + if (replication_handler) + replication_handler->shutdown(); +} + + +template +void DatabasePostgreSQLReplica::loadStoredObjects( + Context & context, bool has_force_restore_data_flag, bool force_attach) +{ + Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach); + startSynchronization(); + +} + + +template +StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, const Context & context) const +{ + if (context.hasQueryContext()) + { + auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; + if (storage_set.find("ReplacingMergeTree") != storage_set.end()) + { + return Base::tryGetTable(name, context); + } + } + + auto table = tables.find(name); + if (table != tables.end()) + return table->second; + + return StoragePtr{}; + +} + + +/// TODO: assert called from sync thread +template +void DatabasePostgreSQLReplica::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +{ + Base::createTable(context, name, table, query); +} + + +template +void DatabasePostgreSQLReplica::dropTable(const Context & context, const String & name, bool no_delay) +{ + Base::dropTable(context, name, no_delay); +} + + +template +void DatabasePostgreSQLReplica::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) +{ + Base::attachTable(name, table, relative_table_path); +} + + +template +StoragePtr DatabasePostgreSQLReplica::detachTable(const String & name) +{ + return Base::detachTable(name); +} + + +template +void DatabasePostgreSQLReplica::drop(const Context & context) +{ + if (replication_handler) + { + replication_handler->shutdown(); + replication_handler->shutdownFinal(); + } + + /// Remove metadata + Poco::File metadata(Base::getMetadataPath() + METADATA_SUFFIX); + + if (metadata.exists()) + metadata.remove(false); + + Base::drop(context); +} + + +template +DatabaseTablesIteratorPtr DatabasePostgreSQLReplica::getTablesIterator( + const Context & /* context */, const DatabaseOnDisk::FilterByNameFunction & /* filter_by_table_name */) +{ + Tables nested_tables; + for (const auto & [table_name, storage] : tables) + { + auto nested_storage = storage->as()->tryGetNested(); + + if (nested_storage) + nested_tables[table_name] = nested_storage; + } + + return std::make_unique(nested_tables, database_name); +} + +} + +#endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h new file mode 100644 index 00000000000..a73acd7b27b --- /dev/null +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h @@ -0,0 +1,88 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX + +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class Context; +class PostgreSQLConnection; +using PostgreSQLConnectionPtr = std::shared_ptr; + + +template +class DatabasePostgreSQLReplica : public Base +{ + +public: + DatabasePostgreSQLReplica( + const Context & context, + const String & metadata_path_, + UUID uuid, + const ASTStorage * database_engine_define, + const String & dbname_, + const String & postgres_dbname, + PostgreSQLConnectionPtr connection_, + std::unique_ptr settings_); + + String getEngineName() const override { return "PostgreSQLReplica"; } + String getMetadataPath() const override { return metadata_path; } + + void loadStoredObjects(Context &, bool, bool force_attach) override; + + DatabaseTablesIteratorPtr getTablesIterator( + const Context & context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override; + + StoragePtr tryGetTable(const String & name, const Context & context) const override; + + void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + + void dropTable(const Context & context, const String & name, bool no_delay) override; + + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; + + StoragePtr detachTable(const String & name) override; + + void drop(const Context & context) override; + + void shutdown() override; + + +private: + void startSynchronization(); + StoragePtr getStorage(const String & name); + + const Context global_context; + String metadata_path; + ASTPtr database_engine_define; + String database_name, remote_database_name; + PostgreSQLConnectionPtr connection; + std::unique_ptr settings; + + std::shared_ptr replication_handler; + std::map tables; + + bool checkPostgresTable(const String & table_name) const; + std::unordered_set fetchTablesList() const; + StoragePtr fetchTable(const String & table_name, const Context & context, const bool table_checked) const; + void removeOutdatedTables(); + ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; +}; + +} + +#endif diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index ec23cfc8794..28f698b3da5 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -25,6 +25,20 @@ namespace ErrorCodes } +std::unordered_set fetchPostgreSQLTablesList(ConnectionPtr connection) +{ + std::unordered_set tables; + std::string query = "SELECT tablename FROM pg_catalog.pg_tables " + "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'"; + pqxx::read_transaction tx(*connection); + + for (auto table_name : tx.stream(query)) + tables.insert(std::get<0>(table_name)); + + return tables; +} + + static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions) { DataTypePtr res; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index bbbb379541b..a507514e92d 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -11,8 +11,10 @@ namespace DB { +std::unordered_set fetchPostgreSQLTablesList(ConnectionPtr connection); + std::shared_ptr fetchPostgreSQLTableStructure( - std::shared_ptr connection, const String & postgres_table_name, bool use_nulls); + ConnectionPtr connection, const String & postgres_table_name, bool use_nulls); } diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.h b/src/Storages/PostgreSQL/PostgreSQLConnection.h index ae79a3436e0..6ffbfe2d20a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLConnection.h +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.h @@ -16,9 +16,10 @@ namespace DB /// Connection is not made until actually used. class PostgreSQLConnection { - using ConnectionPtr = std::shared_ptr; public: + using ConnectionPtr = std::shared_ptr; + PostgreSQLConnection(std::string dbname, std::string host, UInt16 port, std::string user, std::string password) : connection_str(formatConnectionString(std::move(dbname), std::move(host), port, std::move(user), std::move(password))) {} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 842c5b3a5d5..3435abc1fa9 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -47,21 +47,28 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( buffers.emplace(table_name, BufferData(storage->getInMemoryMetadata().getSampleBlock())); } - wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ replicationStream(); }); + wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ synchronizationStream(); }); wal_reader_task->deactivate(); } void PostgreSQLReplicaConsumer::startSynchronization() { - metadata.readMetadata(); - - if (!metadata.lsn().empty()) + try { - auto tx = std::make_shared(*connection->conn()); - final_lsn = metadata.lsn(); - final_lsn = advanceLSN(tx); - tx->commit(); + metadata.readMetadata(); + + if (!metadata.lsn().empty()) + { + auto tx = std::make_shared(*connection->conn()); + final_lsn = metadata.lsn(); + final_lsn = advanceLSN(tx); + tx->commit(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } wal_reader_task->activateAndSchedule(); @@ -75,11 +82,10 @@ void PostgreSQLReplicaConsumer::stopSynchronization() } -void PostgreSQLReplicaConsumer::replicationStream() +void PostgreSQLReplicaConsumer::synchronizationStream() { auto start_time = std::chrono::steady_clock::now(); - - LOG_TRACE(log, "Starting replication stream"); + LOG_TRACE(log, "Starting synchronization stream"); while (!stop_synchronization) { @@ -89,10 +95,7 @@ void PostgreSQLReplicaConsumer::replicationStream() auto end_time = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); if (duration.count() > max_thread_work_duration_ms) - { - LOG_TRACE(log, "Reschedule replication_stream. Thread work duration limit exceeded."); break; - } } if (!stop_synchronization) @@ -148,7 +151,7 @@ void PostgreSQLReplicaConsumer::readString(const char * message, size_t & pos, s } -Int32 PostgreSQLReplicaConsumer::readInt32(const char * message, size_t & pos) +Int32 PostgreSQLReplicaConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 8); Int32 result = (UInt32(unhex2(message + pos)) << 24) @@ -160,7 +163,7 @@ Int32 PostgreSQLReplicaConsumer::readInt32(const char * message, size_t & pos) } -Int16 PostgreSQLReplicaConsumer::readInt16(const char * message, size_t & pos) +Int16 PostgreSQLReplicaConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 4); Int16 result = (UInt32(unhex2(message + pos)) << 8) @@ -170,7 +173,7 @@ Int16 PostgreSQLReplicaConsumer::readInt16(const char * message, size_t & pos) } -Int8 PostgreSQLReplicaConsumer::readInt8(const char * message, size_t & pos) +Int8 PostgreSQLReplicaConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 2); Int8 result = unhex2(message + pos); @@ -179,7 +182,7 @@ Int8 PostgreSQLReplicaConsumer::readInt8(const char * message, size_t & pos) } -Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos) +Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 16); Int64 result = (UInt64(unhex4(message + pos)) << 48) @@ -192,19 +195,19 @@ Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos) void PostgreSQLReplicaConsumer::readTupleData( - BufferData & buffer, const char * message, size_t & pos, PostgreSQLQuery type, bool old_value) + BufferData & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { - Int16 num_columns = readInt16(message, pos); + Int16 num_columns = readInt16(message, pos, size); /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data LOG_DEBUG(log, "num_columns {}", num_columns); for (int column_idx = 0; column_idx < num_columns; ++column_idx) { - char identifier = readInt8(message, pos); - Int32 col_len = readInt32(message, pos); + char identifier = readInt8(message, pos, size); + Int32 col_len = readInt32(message, pos, size); String value; for (int i = 0; i < col_len; ++i) { - value += readInt8(message, pos); + value += readInt8(message, pos, size); } insertValue(buffer, value, column_idx); @@ -242,32 +245,31 @@ void PostgreSQLReplicaConsumer::readTupleData( } } - -void PostgreSQLReplicaConsumer::processReplicationMessage( - const char * replication_message, size_t size, std::unordered_set & tables_to_sync) +/// test relation id can be shuffled ? +void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replication_message, size_t size) { /// Skip '\x' size_t pos = 2; - char type = readInt8(replication_message, pos); + char type = readInt8(replication_message, pos, size); LOG_TRACE(log, "TYPE: {}", type); switch (type) { case 'B': // Begin { - Int64 transaction_end_lsn = readInt64(replication_message, pos); - Int64 transaction_commit_timestamp = readInt64(replication_message, pos); + Int64 transaction_end_lsn = readInt64(replication_message, pos, size); + Int64 transaction_commit_timestamp = readInt64(replication_message, pos, size); LOG_DEBUG(log, "transaction lsn {}, transaction commit timespamp {}", transaction_end_lsn, transaction_commit_timestamp); break; } case 'C': // Commit { - readInt8(replication_message, pos); - Int64 commit_lsn = readInt64(replication_message, pos); - Int64 transaction_end_lsn = readInt64(replication_message, pos); + readInt8(replication_message, pos, size); + Int64 commit_lsn = readInt64(replication_message, pos, size); + Int64 transaction_end_lsn = readInt64(replication_message, pos, size); /// Since postgres epoch - Int64 transaction_commit_timestamp = readInt64(replication_message, pos); + Int64 transaction_commit_timestamp = readInt64(replication_message, pos, size); LOG_DEBUG(log, "commit lsn {}, transaction lsn {}, transaction commit timestamp {}", commit_lsn, transaction_end_lsn, transaction_commit_timestamp); final_lsn = current_lsn; @@ -277,12 +279,17 @@ void PostgreSQLReplicaConsumer::processReplicationMessage( break; case 'R': // Relation { - Int32 relation_id = readInt32(replication_message, pos); + Int32 relation_id = readInt32(replication_message, pos, size); String relation_namespace, relation_name; readString(replication_message, pos, size, relation_namespace); + readString(replication_message, pos, size, relation_name); - Int8 replica_identity = readInt8(replication_message, pos); - Int16 num_columns = readInt16(replication_message, pos); + table_to_insert = relation_name; + tables_to_sync.insert(table_to_insert); + LOG_DEBUG(log, "INSERTING TABLE {}", table_to_insert); + + Int8 replica_identity = readInt8(replication_message, pos, size); + Int16 num_columns = readInt16(replication_message, pos, size); LOG_DEBUG(log, "Replication message type 'R', relation_id: {}, namespace: {}, relation name {}, replica identity {}, columns number {}", @@ -293,14 +300,13 @@ void PostgreSQLReplicaConsumer::processReplicationMessage( for (uint16_t i = 0; i < num_columns; ++i) { String column_name; - key = readInt8(replication_message, pos); + key = readInt8(replication_message, pos, size); readString(replication_message, pos, size, column_name); - data_type_id = readInt32(replication_message, pos); - type_modifier = readInt32(replication_message, pos); + data_type_id = readInt32(replication_message, pos, size); + type_modifier = readInt32(replication_message, pos, size); LOG_DEBUG(log, "Key {}, column name {}, data type id {}, type modifier {}", key, column_name, data_type_id, type_modifier); } - table_to_insert = relation_name; if (storages.find(table_to_insert) == storages.end()) { throw Exception(ErrorCodes::UNKNOWN_TABLE, @@ -309,57 +315,55 @@ void PostgreSQLReplicaConsumer::processReplicationMessage( [[maybe_unused]] auto buffer_iter = buffers.find(table_to_insert); assert(buffer_iter != buffers.end()); - tables_to_sync.insert(relation_name); - break; } case 'Y': // Type break; case 'I': // Insert { - Int32 relation_id = readInt32(replication_message, pos); - Int8 new_tuple = readInt8(replication_message, pos); + Int32 relation_id = readInt32(replication_message, pos, size); + Int8 new_tuple = readInt8(replication_message, pos, size); - LOG_DEBUG(log, "relationID {}, newTuple {}", relation_id, new_tuple); + LOG_DEBUG(log, "relationID {}, newTuple {} current insert tabel {}", relation_id, new_tuple, table_to_insert); auto buffer = buffers.find(table_to_insert); if (buffer == buffers.end()) { throw Exception(ErrorCodes::UNKNOWN_TABLE, "Buffer for table {} does not exist", table_to_insert); } - readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::INSERT); + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT); break; } case 'U': // Update { - Int32 relation_id = readInt32(replication_message, pos); - Int8 primary_key_or_old_tuple_data = readInt8(replication_message, pos); + Int32 relation_id = readInt32(replication_message, pos, size); + Int8 primary_key_or_old_tuple_data = readInt8(replication_message, pos, size); - LOG_DEBUG(log, "relationID {}, key {}", relation_id, primary_key_or_old_tuple_data); + LOG_DEBUG(log, "relationID {}, key {} current insert table {}", relation_id, primary_key_or_old_tuple_data, table_to_insert); auto buffer = buffers.find(table_to_insert); - readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::UPDATE, true); + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); if (pos + 1 < size) { - Int8 new_tuple_data = readInt8(replication_message, pos); + Int8 new_tuple_data = readInt8(replication_message, pos, size); LOG_DEBUG(log, "new tuple data {}", new_tuple_data); - readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::UPDATE); + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE); } break; } case 'D': // Delete { - Int32 relation_id = readInt32(replication_message, pos); + Int32 relation_id = readInt32(replication_message, pos, size); //Int8 index_replica_identity = readInt8(replication_message, pos); - Int8 full_replica_identity = readInt8(replication_message, pos); + Int8 full_replica_identity = readInt8(replication_message, pos, size); LOG_DEBUG(log, "relationID {}, full replica identity {}", relation_id, full_replica_identity); auto buffer = buffers.find(table_to_insert); - readTupleData(buffer->second, replication_message, pos, PostgreSQLQuery::DELETE); + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE); break; } case 'T': // Truncate @@ -371,54 +375,64 @@ void PostgreSQLReplicaConsumer::processReplicationMessage( } -void PostgreSQLReplicaConsumer::syncTables( - std::shared_ptr tx, const std::unordered_set & tables_to_sync) +void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr tx) { + LOG_TRACE(log, "AVAILABLE TABLES {}", tables_to_sync.size()); for (const auto & table_name : tables_to_sync) { - auto & buffer = buffers.find(table_name)->second; - Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns)); - - if (result_rows.rows()) + try { - LOG_TRACE(log, "SYNCING TABLE {} max_block_size {}", result_rows.rows(), max_block_size); + LOG_TRACE(log, "ATTEMPT SYNCING TABLE {}", table_name); + auto & buffer = buffers.find(table_name)->second; + Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns)); - metadata.commitMetadata(final_lsn, [&]() + if (result_rows.rows()) { - Context insert_context(*context); - insert_context.makeQueryContext(); + LOG_TRACE(log, "SYNCING TABLE {} rows {} max_block_size {}", table_name, result_rows.rows(), max_block_size); - auto insert = std::make_shared(); - insert->table_id = storages[table_name]->getStorageID(); + metadata.commitMetadata(final_lsn, [&]() + { + Context insert_context(*context); + insert_context.makeQueryContext(); + insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - InterpreterInsertQuery interpreter(insert, insert_context); - auto block_io = interpreter.execute(); + auto insert = std::make_shared(); + insert->table_id = storages[table_name]->getStorageID(); - /// TODO: what if one block is not enough - OneBlockInputStream input(result_rows); + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); - copyData(input, *block_io.out); + /// TODO: what if one block is not enough + OneBlockInputStream input(result_rows); - LOG_TRACE(log, "TABLE SYNC END"); + copyData(input, *block_io.out); - auto actual_lsn = advanceLSN(tx); - buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); + LOG_TRACE(log, "TABLE SYNC END"); - return actual_lsn; - }); + auto actual_lsn = advanceLSN(tx); + buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); + + return actual_lsn; + }); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } } + + tables_to_sync.clear(); + tx->commit(); } -String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr ntx) +String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr tx) { LOG_TRACE(log, "CURRENT LSN FROM TO {}", final_lsn); std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn); - pqxx::result result{ntx->exec(query_str)}; - - ntx->commit(); + pqxx::result result{tx->exec(query_str)}; if (!result.empty()) return result[0][0].as(); @@ -432,7 +446,6 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { std::shared_ptr tx; bool slot_empty = true; - std::unordered_set tables_to_sync; try { @@ -471,20 +484,18 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() current_lsn = (*row)[0]; LOG_TRACE(log, "Replication message: {}", (*row)[1]); - processReplicationMessage((*row)[1].c_str(), (*row)[1].size(), tables_to_sync); + processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } } catch (const pqxx::sql_error & e) { - /// Currently `sql replication interface` is used and it has the problem that it registers relcache + /// sql replication interface has the problem that it registers relcache /// callbacks on each pg_logical_slot_get_changes and there is no way to invalidate them: /// https://github.com/postgres/postgres/blob/master/src/backend/replication/pgoutput/pgoutput.c#L1128 /// So at some point will get out of limit and then they will be cleaned. std::string error_message = e.what(); - if (error_message.find("out of relcache_callback_list slots") != std::string::npos) - LOG_DEBUG(log, "Out of rel_cache_list slot"); - else + if (error_message.find("out of relcache_callback_list slots") == std::string::npos) tryLogCurrentException(__PRETTY_FUNCTION__); return false; @@ -498,8 +509,7 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() return false; } - syncTables(tx, tables_to_sync); - + syncTables(tx); return true; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 817c57a99fa..0973ba7f785 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -29,29 +29,20 @@ public: const size_t max_block_size_, Storages storages_); - /// Start reading WAL from current_lsn position. Initial data sync from created snapshot already done. void startSynchronization(); + void stopSynchronization(); private: - /// Executed by wal_reader_task. A separate thread reads wal and advances lsn to last commited position - /// after rows were written via copyData. - void replicationStream(); - void stopReplicationStream(); - - enum class PostgreSQLQuery - { - INSERT, - UPDATE, - DELETE - }; + void synchronizationStream(); bool readFromReplicationSlot(); - void syncTables(std::shared_ptr tx, const std::unordered_set & tables_to_sync); + + void syncTables(std::shared_ptr tx); + String advanceLSN(std::shared_ptr ntx); - void processReplicationMessage( - const char * replication_message, size_t size, std::unordered_set & tables_to_sync); + void processReplicationMessage(const char * replication_message, size_t size); struct BufferData { @@ -74,26 +65,34 @@ private: void insertDefaultValue(BufferData & buffer, size_t column_idx); void insertValue(BufferData & buffer, const std::string & value, size_t column_idx); - void readTupleData(BufferData & buffer, const char * message, size_t & pos, PostgreSQLQuery type, bool old_value = false); - /// Methods to parse replication message data. + enum class PostgreSQLQuery + { + INSERT, + UPDATE, + DELETE + }; + + void readTupleData(BufferData & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); + void readString(const char * message, size_t & pos, size_t size, String & result); - Int64 readInt64(const char * message, size_t & pos); - Int32 readInt32(const char * message, size_t & pos); - Int16 readInt16(const char * message, size_t & pos); - Int8 readInt8(const char * message, size_t & pos); + Int64 readInt64(const char * message, size_t & pos, size_t size); + Int32 readInt32(const char * message, size_t & pos, size_t size); + Int16 readInt16(const char * message, size_t & pos, size_t size); + Int8 readInt8(const char * message, size_t & pos, size_t size); Poco::Logger * log; std::shared_ptr context; - const std::string replication_slot_name; - const std::string publication_name; - PostgreSQLReplicaMetadata metadata; + const std::string replication_slot_name, publication_name; + PostgreSQLReplicaMetadata metadata; PostgreSQLConnectionPtr connection; std::string current_lsn, final_lsn; const size_t max_block_size; + std::string table_to_insert; + std::unordered_set tables_to_sync; BackgroundSchedulePool::TaskHolder wal_reader_task; std::atomic stop_synchronization = false; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index a5ae25c3f53..9cd5f368a6d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -27,6 +27,7 @@ PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadat void PostgreSQLReplicaMetadata::readMetadata() { + LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaMetadata"), "kssenii 1 {}", metadata_file); if (Poco::File(metadata_file).exists()) { ReadBufferFromFile in(metadata_file, DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h index 31044dc3490..ca7a258e24c 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h @@ -11,9 +11,11 @@ public: PostgreSQLReplicaMetadata(const std::string & metadata_file_path); void commitMetadata(std::string & lsn, const std::function & syncTableFunc); + void readMetadata(); size_t version() { return last_version++; } + std::string lsn() { return last_lsn; } private: diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index e9d7b1d9a69..c803f0aa411 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,7 +1,11 @@ #include "PostgreSQLReplicationHandler.h" -#include #include +#include +#include + +#include +#include #include #include @@ -18,6 +22,8 @@ namespace ErrorCodes static const auto reschedule_ms = 500; +/// TODO: context should be const + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const std::string & conn_str, @@ -45,9 +51,9 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } -void PostgreSQLReplicationHandler::addStoragePtr(const std::string & table_name, StoragePtr storage) +void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, const StoragePostgreSQLReplica * storage) { - storages[table_name] = std::move(storage); + storages[table_name] = storage; } @@ -61,23 +67,22 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { try { + /// Will throw pqxx::broken_connection if no connection at the moment connection->conn(); + + startSynchronization(); } catch (const pqxx::broken_connection & pqxx_error) { - LOG_ERROR(log, - "Unable to set up connection. Reconnection attempt continue. Error message: {}", + LOG_ERROR(log, "Unable to set up connection. Reconnection attempt continue. Error message: {}", pqxx_error.what()); startup_task->scheduleAfter(reschedule_ms); } - catch (Exception & e) + catch (...) { - e.addMessage("while setting up connection for PostgreSQLReplica engine"); - throw; + tryLogCurrentException(__PRETTY_FUNCTION__); } - - startReplication(); } @@ -118,7 +123,7 @@ void PostgreSQLReplicationHandler::createPublication(std::shared_ptr try { tx->exec(query_str); - LOG_TRACE(log, "Created publication {}", publication_name); + LOG_TRACE(log, "Created publication {} with tables list: {}", publication_name, table_names); } catch (Exception & e) { @@ -131,10 +136,8 @@ void PostgreSQLReplicationHandler::createPublication(std::shared_ptr } -void PostgreSQLReplicationHandler::startReplication() +void PostgreSQLReplicationHandler::startSynchronization() { - LOG_DEBUG(log, "PostgreSQLReplica starting replication proccess"); - /// used commands require a specific transaction isolation mode. replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); @@ -177,10 +180,17 @@ void PostgreSQLReplicationHandler::startReplication() dropReplicationSlot(ntx, replication_slot); initial_sync(); } + else + { + for (const auto & [table_name, storage] : storages) + { + nested_storages[table_name] = storage->getNested(); + storage->setNestedLoaded(); + } + } ntx->commit(); - LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Creating replication consumer"); consumer = std::make_shared( context, std::move(connection), @@ -189,13 +199,10 @@ void PostgreSQLReplicationHandler::startReplication() metadata_path, start_lsn, max_block_size, - storages); - - LOG_DEBUG(&Poco::Logger::get("StoragePostgreSQLMetadata"), "Successfully created replication consumer"); + nested_storages); consumer->startSynchronization(); - /// Takes time to close replication_connection->conn()->close(); } @@ -206,6 +213,12 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) for (const auto & [table_name, storage] : storages) { + storage->createNestedIfNeeded(); + auto nested_storage = storage->tryGetNested(); + + if (!nested_storage) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to create nested storage"); + try { auto stx = std::make_unique(*connection->conn()); @@ -221,19 +234,23 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) Context insert_context(*context); insert_context.makeQueryContext(); + insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); auto insert = std::make_shared(); - insert->table_id = storage->getStorageID(); + insert->table_id = nested_storage->getStorageID(); InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); - const StorageInMemoryMetadata & storage_metadata = storage->getInMemoryMetadata(); + const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); PostgreSQLBlockInputStream input(std::move(stx), query_str, sample_block, DEFAULT_BLOCK_SIZE); copyData(input, *block_io.out); + + storage->setNestedLoaded(); + nested_storages[table_name] = nested_storage; } catch (Exception & e) { @@ -271,7 +288,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx, pqxx::result result{ntx->exec(query_str)}; start_lsn = result[0][1].as(); snapshot_name = result[0][2].as(); - LOG_TRACE(log, "Created temporary replication slot: {}, start lsn: {}, snapshot: {}", + LOG_TRACE(log, "Created replication slot: {}, start lsn: {}, snapshot: {}", replication_slot, start_lsn, snapshot_name); } catch (Exception & e) @@ -286,7 +303,7 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, st { std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); ntx->exec(query_str); - LOG_TRACE(log, "Replication slot {} is dropped", slot_name); + LOG_TRACE(log, "Dropped replication slot {}", slot_name); } @@ -306,13 +323,50 @@ void PostgreSQLReplicationHandler::shutdownFinal() Poco::File(metadata_path).remove(); connection = std::make_shared(connection_str); - auto ntx = std::make_shared(*connection->conn()); + auto tx = std::make_shared(*connection->conn()); - dropPublication(ntx); - if (isReplicationSlotExist(ntx, replication_slot)) - dropReplicationSlot(ntx, replication_slot); + dropPublication(tx); + if (isReplicationSlotExist(tx, replication_slot)) + dropReplicationSlot(tx, replication_slot); - ntx->commit(); + tx->commit(); +} + + +/// TODO: publication can be created with option `whole_database`. Check this case. +std::unordered_set PostgreSQLReplicationHandler::fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_) +{ + auto publication_exist = [&]() + { + auto tx = std::make_shared(*connection_); + bool exist = isPublicationExist(tx); + tx->commit(); + return exist; + }; + + if (publication_name.empty() || !publication_exist()) + { + /// Replicate the whole database and create our own pg_publication + return fetchPostgreSQLTablesList(connection_); + } + else + { + /// Replicate only tables, which are included in a pg_publication + return fetchTablesFromPublication(connection_); + } +} + + +std::unordered_set PostgreSQLReplicationHandler::fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_) +{ + std::string query = fmt::format("SELECT tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); + std::unordered_set tables; + pqxx::read_transaction tx(*connection_); + + for (auto table_name : tx.stream(query)) + tables.insert(std::get<0>(table_name)); + + return tables; } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index f4118c4aed8..ba6014aed1f 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -8,6 +8,8 @@ namespace DB { +class StoragePostgreSQLReplica; + class PostgreSQLReplicationHandler { public: @@ -22,13 +24,17 @@ public: const size_t max_block_size_); void startup(); - void addStoragePtr(const std::string & table_name, StoragePtr storage); + void shutdown(); + void shutdownFinal(); + void addStorage(const std::string & table_name, const StoragePostgreSQLReplica * storage); + + std::unordered_set fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_); + private: using NontransactionPtr = std::shared_ptr; - using Storages = std::unordered_map; bool isPublicationExist(std::shared_ptr tx); bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); @@ -40,9 +46,13 @@ private: void dropPublication(NontransactionPtr ntx); void waitConnectionAndStart(); - void startReplication(); + + void startSynchronization(); + void loadFromSnapshot(std::string & snapshot_name); + std::unordered_set fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_); + Poco::Logger * log; std::shared_ptr context; const std::string database_name, connection_str, metadata_path; @@ -53,8 +63,10 @@ private: std::shared_ptr consumer; BackgroundSchedulePool::TaskHolder startup_task; + std::atomic tables_loaded = false; - Storages storages; + std::unordered_map storages; + std::unordered_map nested_storages; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 27ab6c7adaf..490da9ce322 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -26,6 +27,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree"; @@ -36,20 +38,18 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( const String & remote_database_name, const String & remote_table_name_, const String & connection_str, - const String & relative_data_path_, const StorageInMemoryMetadata & storage_metadata, const Context & context_, std::unique_ptr replication_settings_) : IStorage(table_id_) , remote_table_name(remote_table_name_) - , relative_data_path(relative_data_path_) , global_context(std::make_shared(context_.getGlobalContext())) , replication_settings(std::move(replication_settings_)) { setInMemoryMetadata(storage_metadata); - if (relative_data_path.ends_with("/")) - relative_data_path.resize(relative_data_path.size() - 1); - relative_data_path += NESTED_STORAGE_SUFFIX; + + is_postgresql_replica_database = DatabaseCatalog::instance().getDatabase( + getStorageID().database_name)->getEngineName() == "PostgreSQLReplica"; auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath() + "/.metadata_" + table_id_.database_name + "_" + table_id_.table_name; @@ -69,8 +69,47 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( } +StoragePostgreSQLReplica::StoragePostgreSQLReplica( + const StorageID & table_id_, + const String & /* metadata_path_ */, + const StorageInMemoryMetadata & storage_metadata, + const Context & context_) + : IStorage(table_id_) + , global_context(std::make_shared(context_)) +{ + setInMemoryMetadata(storage_metadata); + is_postgresql_replica_database = DatabaseCatalog::instance().getDatabase( + getStorageID().database_name)->getEngineName() == "PostgreSQLReplica"; +} + + +StoragePostgreSQLReplica::StoragePostgreSQLReplica( + const StorageID & table_id_, + StoragePtr nested_storage_, + const Context & context_) + : IStorage(table_id_) + , global_context(std::make_shared(context_)) + , nested_storage(nested_storage_) +{ + is_postgresql_replica_database = DatabaseCatalog::instance().getDatabase( + getStorageID().database_name)->getEngineName() == "PostgreSQLReplica"; + +} + + +std::string StoragePostgreSQLReplica::getNestedTableName() const +{ + auto table_name = getStorageID().table_name; + + if (!is_postgresql_replica_database) + table_name += NESTED_STORAGE_SUFFIX; + + return table_name; +} + + std::shared_ptr StoragePostgreSQLReplica::getMaterializedColumnsDeclaration( - const String name, const String type, UInt64 default_value) + const String name, const String type, UInt64 default_value) const { auto column_declaration = std::make_shared(); @@ -87,7 +126,7 @@ std::shared_ptr StoragePostgreSQLReplica::getMaterializedC } -ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_type) +ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_type) const { WhichDataType which(data_type); @@ -101,12 +140,13 @@ ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_t } -std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage() +std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage() const { auto columns_declare_list = std::make_shared(); auto columns_expression_list = std::make_shared(); auto metadata_snapshot = getInMemoryMetadataPtr(); + for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) { const auto & column_declaration = std::make_shared(); @@ -114,6 +154,7 @@ std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage( column_declaration->type = getColumnDeclaration(column_type_and_name.type); columns_expression_list->children.emplace_back(column_declaration); } + columns_declare_list->set(columns_declare_list->columns, columns_expression_list); columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", UInt64(1))); @@ -123,14 +164,14 @@ std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage( } -ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() +ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery() const { auto create_table_query = std::make_shared(); auto table_id = getStorageID(); - create_table_query->table = table_id.table_name + NESTED_STORAGE_SUFFIX; + create_table_query->table = getNestedTableName(); create_table_query->database = table_id.database_name; - create_table_query->if_not_exists = true; + //create_table_query->if_not_exists = true; create_table_query->set(create_table_query->columns_list, getColumnsListFromStorage()); @@ -138,8 +179,10 @@ ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared("_version"))); auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST(); - if (primary_key_ast) - storage->set(storage->order_by, primary_key_ast); + if (!primary_key_ast) + primary_key_ast = std::make_shared("key"); + + storage->set(storage->order_by, primary_key_ast); create_table_query->set(create_table_query->storage, storage); @@ -147,6 +190,120 @@ ASTPtr StoragePostgreSQLReplica::getCreateHelperTableQuery() } +void StoragePostgreSQLReplica::createNestedIfNeeded() const +{ + nested_storage = tryGetNested(); + + if (nested_storage) + return; + + Context context_copy(*global_context); + const auto ast_create = getCreateNestedTableQuery(); + + InterpreterCreateQuery interpreter(ast_create, context_copy); + try + { + interpreter.execute(); + } + catch (...) + { + throw; + } + + nested_storage = getNested(); +} + + +Context StoragePostgreSQLReplica::makeGetNestedTableContext() const +{ + auto get_context(*global_context); + get_context.makeQueryContext(); + get_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + + return get_context; +} + + +StoragePtr StoragePostgreSQLReplica::getNested() const +{ + if (nested_storage) + return nested_storage; + + auto context = makeGetNestedTableContext(); + nested_storage = DatabaseCatalog::instance().getTable( + StorageID(getStorageID().database_name, getNestedTableName()), context); + + return nested_storage; +} + + +StoragePtr StoragePostgreSQLReplica::tryGetNested() const +{ + if (nested_storage) + return nested_storage; + + auto context = makeGetNestedTableContext(); + nested_storage = DatabaseCatalog::instance().tryGetTable( + StorageID(getStorageID().database_name, getNestedTableName()), context); + + return nested_storage; +} + + +void StoragePostgreSQLReplica::startup() +{ + if (!is_postgresql_replica_database) + { + replication_handler->addStorage(remote_table_name, this); + replication_handler->startup(); + } +} + + +void StoragePostgreSQLReplica::shutdown() +{ + if (replication_handler) + replication_handler->shutdown(); +} + + +void StoragePostgreSQLReplica::shutdownFinal() +{ + if (is_postgresql_replica_database) + return; + + if (replication_handler) + replication_handler->shutdownFinal(); + + if (nested_storage) + dropNested(); +} + + +void StoragePostgreSQLReplica::dropNested() +{ + auto table_id = nested_storage->getStorageID(); + auto ast_drop = std::make_shared(); + + ast_drop->kind = ASTDropQuery::Drop; + ast_drop->table = table_id.table_name; + ast_drop->database = table_id.database_name; + ast_drop->if_exists = true; + + auto drop_context(*global_context); + drop_context.makeQueryContext(); + + auto interpreter = InterpreterDropQuery(ast_drop, drop_context); + interpreter.execute(); +} + + +NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const +{ + return NamesAndTypesList{}; +} + + Pipe StoragePostgreSQLReplica::read( const Names & column_names, const StorageMetadataPtr & /* metadata_snapshot */, @@ -156,14 +313,17 @@ Pipe StoragePostgreSQLReplica::read( size_t max_block_size, unsigned num_streams) { - if (!nested_storage) + /// If initial table sync has not yet finished, nested tables might not be created yet. + if (!nested_loaded) { - auto table_id = getStorageID(); - nested_storage = DatabaseCatalog::instance().getTable( - StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), - *global_context); + LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Table {} is not loaded yet", getNestedTableName()); + return Pipe(); } + /// Should throw if there is no nested storage + if (!nested_storage) + getNested(); + auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); @@ -227,70 +387,6 @@ Pipe StoragePostgreSQLReplica::read( } -void StoragePostgreSQLReplica::startup() -{ - Context context_copy(*global_context); - const auto ast_create = getCreateHelperTableQuery(); - auto table_id = getStorageID(); - - Poco::File path(relative_data_path); - if (!path.exists()) - { - LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), - "Creating helper table {}", table_id.table_name + NESTED_STORAGE_SUFFIX); - InterpreterCreateQuery interpreter(ast_create, context_copy); - interpreter.execute(); - } - else - LOG_TRACE(&Poco::Logger::get("StoragePostgreSQLReplica"), - "Directory already exists {}", relative_data_path); - - replication_handler->addStoragePtr( - remote_table_name, - DatabaseCatalog::instance().getTable( - StorageID(table_id.database_name, table_id.table_name + NESTED_STORAGE_SUFFIX), *global_context)); - - replication_handler->startup(); -} - - -void StoragePostgreSQLReplica::shutdown() -{ - replication_handler->shutdown(); -} - - -void StoragePostgreSQLReplica::shutdownFinal() -{ - replication_handler->shutdownFinal(); - dropNested(); -} - - -void StoragePostgreSQLReplica::dropNested() -{ - auto table_id = nested_storage->getStorageID(); - auto ast_drop = std::make_shared(); - - ast_drop->kind = ASTDropQuery::Drop; - ast_drop->table = table_id.table_name; - ast_drop->database = table_id.database_name; - ast_drop->if_exists = true; - - auto drop_context(*global_context); - drop_context.makeQueryContext(); - - auto interpreter = InterpreterDropQuery(ast_drop, drop_context); - interpreter.execute(); -} - - -NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const -{ - return NamesAndTypesList{}; -} - - void registerStoragePostgreSQLReplica(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) @@ -339,7 +435,7 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) return StoragePostgreSQLReplica::create( args.table_id, remote_database, remote_table, connection.conn_str(), - args.relative_data_path, metadata, args.context, + metadata, args.context, std::move(postgresql_replication_settings)); }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index c1e4b319187..277d6fc9313 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -26,6 +26,17 @@ class StoragePostgreSQLReplica final : public ext::shared_ptr_helper; public: + StoragePostgreSQLReplica( + const StorageID & table_id_, + const String & metadata_path_, + const StorageInMemoryMetadata & storage_metadata, + const Context & context_); + + StoragePostgreSQLReplica( + const StorageID & table_id_, + StoragePtr nested_storage_, + const Context & context_); + String getName() const override { return "PostgreSQLReplica"; } void startup() override; @@ -45,33 +56,52 @@ public: /// Called right after shutdown() in case of drop query void shutdownFinal(); + void createNestedIfNeeded() const; + + /// Can be nullptr + StoragePtr tryGetNested() const; + + /// Throw if impossible to get + StoragePtr getNested() const; + + void setNestedLoaded() const { nested_loaded.store(true); } + protected: StoragePostgreSQLReplica( const StorageID & table_id_, const String & remote_database_name, const String & remote_table_name, const String & connection_str, - const String & relative_data_path_, const StorageInMemoryMetadata & storage_metadata, const Context & context_, std::unique_ptr replication_settings_); private: std::shared_ptr getMaterializedColumnsDeclaration( - const String name, const String type, UInt64 default_value); - std::shared_ptr getColumnsListFromStorage(); - ASTPtr getColumnDeclaration(const DataTypePtr & data_type); - ASTPtr getCreateHelperTableQuery(); + const String name, const String type, UInt64 default_value) const; + + std::shared_ptr getColumnsListFromStorage() const; + + ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; + + ASTPtr getCreateNestedTableQuery() const; + + std::string getNestedTableName() const; + + Context makeGetNestedTableContext() const; + void dropNested(); - std::string remote_table_name, relative_data_path; + std::string remote_table_name; std::shared_ptr global_context; std::unique_ptr replication_settings; std::unique_ptr replication_handler; - /// ReplacingMergeTree table - StoragePtr nested_storage; + bool is_postgresql_replica_database = false; + + mutable std::atomic nested_loaded = false; + mutable StoragePtr nested_storage; }; } diff --git a/tests/integration/test_postgresql_replica_database_engine/__init__.py b/tests/integration/test_postgresql_replica_database_engine/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml b/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml new file mode 100644 index 00000000000..f9d15e572aa --- /dev/null +++ b/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py new file mode 100644 index 00000000000..7d5c36a83c9 --- /dev/null +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -0,0 +1,138 @@ +import pytest +import time +import psycopg2 +import os.path as p + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True) + +postgres_table_template = """ + CREATE TABLE IF NOT EXISTS {} ( + key Integer NOT NULL, value Integer) + """ + +def get_postgres_conn(database=False): + if database == True: + conn_string = "host='localhost' dbname='postgres_database' user='postgres' password='mysecretpassword'" + else: + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + + +def create_postgres_db(cursor, name): + cursor.execute("CREATE DATABASE {}".format(name)) + + +def create_postgres_table(cursor, table_name): + cursor.execute(postgres_table_template.format(table_name)) + cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) + + +def check_tables_are_synchronized(table_name): + expected = instance.query('select * from postgres_database.{} order by key;'.format(table_name)) + result = instance.query('select * from test_database.{} order by key;'.format(table_name)) + + while result != expected: + time.sleep(0.5) + result = instance.query('select * from test_database.{} order by key;'.format(table_name)) + + assert(result == expected) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + conn = get_postgres_conn() + cursor = conn.cursor() + create_postgres_db(cursor, 'postgres_database') + instance.query("DROP DATABASE IF EXISTS test_database") + instance.query(''' + CREATE DATABASE postgres_database + ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')''') + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def rabbitmq_setup_teardown(): + yield # run test + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') + + +def test_load_and_sync_all_database(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 5 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(50)".format(i)) + + instance.query("CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + assert 'test_database' in instance.query('SHOW DATABASES') + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + cursor.execute('drop table postgresql_replica_{};'.format(i)) + + result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''') + assert(int(result) == NUM_TABLES) + + instance.query("DROP DATABASE test_database") + assert 'test_database' not in instance.query('SHOW DATABASES') + + +def test_replicating_dml(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 5 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i)) + + instance.query( + "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + + for i in range(NUM_TABLES): + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50, 50)".format(i, i)) + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + cursor.execute('UPDATE postgresql_replica_{} SET value = {} * {} WHERE key < 50;'.format(i, i, i)) + cursor.execute('UPDATE postgresql_replica_{} SET value = {} * {} * {} WHERE key >= 50;'.format(i, i, i, i)) + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + cursor.execute('DELETE FROM postgresql_replica_{} WHERE (value*value + {}) % 2 = 0;'.format(i, i)) + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + cursor.execute('drop table postgresql_replica_{};'.format(i)) + + instance.query("DROP DATABASE test_database") + assert 'test_database' not in instance.query('SHOW DATABASES') + + +if __name__ == '__main__': + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() From 5bc0010f94178bea02d7b7eece124b3894aaeafc Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 11 Feb 2021 21:59:58 +0000 Subject: [PATCH 015/105] Better --- .../PostgreSQLBlockInputStream.cpp | 48 +++++-- src/DataStreams/PostgreSQLBlockInputStream.h | 13 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 15 +- .../fetchPostgreSQLTableStructure.cpp | 136 +++++++++++++----- .../fetchPostgreSQLTableStructure.h | 18 ++- .../PostgreSQLDictionarySource.cpp | 22 +-- .../PostgreSQLReplicationHandler.cpp | 40 +++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 7 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 131 ++++++++++------- .../PostgreSQL/StoragePostgreSQLReplica.h | 25 ++-- src/Storages/StoragePostgreSQL.cpp | 8 +- .../TableFunctionPostgreSQL.cpp | 7 +- .../test.py | 6 +- 14 files changed, 300 insertions(+), 178 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 5b43a21c6fc..9e3c6b1bb89 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -22,32 +22,55 @@ namespace DB { -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( - std::unique_ptr tx_, + +template<> +PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( + std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, - const UInt64 max_block_size_) + const UInt64 max_block_size_, + bool auto_commit_) : query_str(query_str_) , max_block_size(max_block_size_) - , tx(std::move(tx_)) + , auto_commit(auto_commit_) + , tx(tx_) { description.init(sample_block); +} + + +template<> +PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( + std::shared_ptr tx_, + const std::string & query_str_, + const Block & sample_block, + const UInt64 max_block_size_, + bool auto_commit_) + : query_str(query_str_) + , max_block_size(max_block_size_) + , auto_commit(auto_commit_) + , tx(tx_) +{ + description.init(sample_block); +} + + +template +void PostgreSQLBlockInputStream::readPrefix() +{ for (const auto idx : ext::range(0, description.sample_block.columns())) if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); /// pqxx::stream_from uses COPY command, will get error if ';' is present if (query_str.ends_with(';')) query_str.resize(query_str.size() - 1); -} - -void PostgreSQLBlockInputStream::readPrefix() -{ stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } -Block PostgreSQLBlockInputStream::readImpl() +template +Block PostgreSQLBlockInputStream::readImpl() { /// Check if pqxx::stream_from is finished if (!stream || !(*stream)) @@ -103,12 +126,15 @@ Block PostgreSQLBlockInputStream::readImpl() } -void PostgreSQLBlockInputStream::readSuffix() +template +void PostgreSQLBlockInputStream::readSuffix() { if (stream) { stream->complete(); - tx->commit(); + + if (auto_commit) + tx->commit(); } } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index f51526b2eb3..a558a46d153 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -11,19 +11,22 @@ #include #include #include +#include namespace DB { +template class PostgreSQLBlockInputStream : public IBlockInputStream { public: PostgreSQLBlockInputStream( - std::unique_ptr tx_, - const std::string & query_str, + std::shared_ptr tx_, + const std::string & query_str_, const Block & sample_block, - const UInt64 max_block_size_); + const UInt64 max_block_size_, + bool auto_commit_ = true); String getName() const override { return "PostgreSQL"; } Block getHeader() const override { return description.sample_block.cloneEmpty(); } @@ -35,9 +38,11 @@ private: String query_str; const UInt64 max_block_size; + const bool auto_commit; ExternalResultDescription description; - std::unique_ptr tx; + PostgreSQLConnection::ConnectionPtr connection; + std::shared_ptr tx; std::unique_ptr stream; std::unordered_map array_info; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index ebe5ba107bd..c226540cedc 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -148,7 +148,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte return StoragePtr{}; auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - auto columns = fetchPostgreSQLTableStructure(connection->conn(), table_name, use_nulls); + auto columns = fetchPostgreSQLTableStructure(connection->conn(), table_name, use_nulls).columns; if (!columns) return StoragePtr{}; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index d279d7e5c5c..a9b07a0c65b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -130,18 +130,7 @@ StoragePtr DatabasePostgreSQLReplica::getStorage(const String & name) if (storage) return storage; - auto use_nulls = global_context.getSettingsRef().external_table_functions_use_nulls; - auto columns = fetchPostgreSQLTableStructure(connection->conn(), name, use_nulls); - - if (!columns) - return StoragePtr{}; - - StorageInMemoryMetadata metadata; - metadata.setColumns(ColumnsDescription(*columns)); - - storage = StoragePostgreSQLReplica::create(StorageID(database_name, name), metadata_path, metadata, global_context); - - return storage; + return StoragePostgreSQLReplica::create(StorageID(database_name, name), StoragePtr{}, global_context); } @@ -176,7 +165,7 @@ StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, con } auto table = tables.find(name); - if (table != tables.end()) + if (table != tables.end() && table->second->as()->isNestedLoaded()) return table->second; return StoragePtr{}; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 28f698b3da5..a3624236f76 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -14,6 +14,8 @@ #include #include +#include + namespace DB { @@ -25,7 +27,7 @@ namespace ErrorCodes } -std::unordered_set fetchPostgreSQLTablesList(ConnectionPtr connection) +std::unordered_set fetchPostgreSQLTablesList(PostgreSQLConnection::ConnectionPtr connection) { std::unordered_set tables; std::string query = "SELECT tablename FROM pg_catalog.pg_tables " @@ -39,7 +41,7 @@ std::unordered_set fetchPostgreSQLTablesList(ConnectionPtr connecti } -static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions) +static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable = false, uint16_t dimensions = 0) { DataTypePtr res; @@ -94,16 +96,66 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl } -std::shared_ptr fetchPostgreSQLTableStructure( - std::shared_ptr connection, const String & postgres_table_name, bool use_nulls) +template +std::shared_ptr readNamesAndTypesList( + std::shared_ptr tx, const String & postgres_table_name, const String & query, bool use_nulls, bool only_names_and_types) { auto columns = NamesAndTypesList(); + try + { + pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query)); + + if (only_names_and_types) + { + std::tuple row; + while (stream >> row) + columns.push_back(NameAndTypePair(std::get<0>(row), convertPostgreSQLDataType(std::get<1>(row)))); + } + else + { + std::tuple row; + while (stream >> row) + { + columns.push_back(NameAndTypePair( + std::get<0>(row), /// column name + convertPostgreSQLDataType( + std::get<1>(row), /// data type + use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false + std::get<3>(row)))); /// number of dimensions if data type is array + } + } + + stream.complete(); + } + catch (const pqxx::undefined_table &) + { + throw Exception(fmt::format( + "PostgreSQL table {} does not exist", postgres_table_name), ErrorCodes::UNKNOWN_TABLE); + } + catch (Exception & e) + { + e.addMessage("while fetching postgresql table structure"); + throw; + } + + return !columns.empty() ? std::make_shared(columns) : nullptr; +} + + +template +PostgreSQLTableStructure fetchPostgreSQLTableStructureImpl( + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) +{ + PostgreSQLTableStructure table; + if (postgres_table_name.find('\'') != std::string::npos || postgres_table_name.find('\\') != std::string::npos) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", - postgres_table_name); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", + postgres_table_name); } std::string query = fmt::format( @@ -112,42 +164,52 @@ std::shared_ptr fetchPostgreSQLTableStructure( "FROM pg_attribute " "WHERE attrelid = '{}'::regclass " "AND NOT attisdropped AND attnum > 0", postgres_table_name); - try - { - pqxx::read_transaction tx(*connection); - pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); - std::tuple row; - while (stream >> row) - { - columns.push_back(NameAndTypePair( - std::get<0>(row), - convertPostgreSQLDataType( - std::get<1>(row), - use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false, i.e. value is nullable - std::get<3>(row)))); - } - stream.complete(); - tx.commit(); - } - catch (const pqxx::undefined_table &) - { - throw Exception(fmt::format( - "PostgreSQL table {}.{} does not exist", - connection->dbname(), postgres_table_name), ErrorCodes::UNKNOWN_TABLE); - } - catch (Exception & e) - { - e.addMessage("while fetching postgresql table structure"); - throw; - } + table.columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, false); - if (columns.empty()) - return nullptr; + if (!with_primary_key) + return table; - return std::make_shared(columns); + /// wiki.postgresql.org/wiki/Retrieve_primary_key_columns + query = fmt::format( + "SELECT a.attname, format_type(a.atttypid, a.atttypmod) AS data_type " + "FROM pg_index i " + "JOIN pg_attribute a ON a.attrelid = i.indrelid " + "AND a.attnum = ANY(i.indkey) " + "WHERE i.indrelid = '{}'::regclass AND i.indisprimary", postgres_table_name); + + table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); + + return table; } + +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) +{ + return fetchPostgreSQLTableStructureImpl(tx, postgres_table_name, use_nulls, with_primary_key); +} + + +/// For the case when several operations are made on the transaction object before it can be used (like export snapshot and isolation level) +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) +{ + return fetchPostgreSQLTableStructureImpl(tx, postgres_table_name, use_nulls, with_primary_key); +} + + +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + PostgreSQLConnection::ConnectionPtr connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key) +{ + auto tx = std::make_shared(*connection); + auto table = fetchPostgreSQLTableStructure(tx, postgres_table_name, use_nulls, with_primary_key); + tx->commit(); + + return table; +} + + } #endif diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index a507514e92d..bf3c8ead422 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -5,16 +5,26 @@ #endif #if USE_LIBPQXX -#include +#include +#include namespace DB { -std::unordered_set fetchPostgreSQLTablesList(ConnectionPtr connection); +std::unordered_set fetchPostgreSQLTablesList(PostgreSQLConnection::ConnectionPtr connection); -std::shared_ptr fetchPostgreSQLTableStructure( - ConnectionPtr connection, const String & postgres_table_name, bool use_nulls); +struct PostgreSQLTableStructure +{ + std::shared_ptr columns; + std::shared_ptr primary_key_columns; +}; + +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + PostgreSQLConnection::ConnectionPtr connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); + +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); } diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 8ede0bc8813..954e5f4e187 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -68,9 +68,8 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); - auto tx = std::make_unique(*connection->conn()); - return std::make_shared( - std::move(tx), load_all_query, sample_block, max_block_size); + auto tx = std::make_shared(*connection->conn()); + return std::make_shared>(tx, load_all_query, sample_block, max_block_size); } @@ -78,23 +77,23 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); - auto tx = std::make_unique(*connection->conn()); - return std::make_shared(std::move(tx), load_update_query, sample_block, max_block_size); + auto tx = std::make_shared(*connection->conn()); + return std::make_shared>(tx, load_update_query, sample_block, max_block_size); } BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); - auto tx = std::make_unique(*connection->conn()); - return std::make_shared(std::move(tx), query, sample_block, max_block_size); + auto tx = std::make_shared(*connection->conn()); + return std::make_shared>(tx, query, sample_block, max_block_size); } BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); - auto tx = std::make_unique(*connection->conn()); - return std::make_shared(std::move(tx), query, sample_block, max_block_size); + auto tx = std::make_shared(*connection->conn()); + return std::make_shared>(tx, query, sample_block, max_block_size); } @@ -116,8 +115,9 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - auto tx = std::make_unique(*connection->conn()); - PostgreSQLBlockInputStream block_input_stream(std::move(tx), request, invalidate_sample_block, 1); + auto tx = std::make_shared(*connection->conn()); + PostgreSQLBlockInputStream block_input_stream(tx, request, invalidate_sample_block, 1); + return readInvalidateQuery(block_input_stream); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index c803f0aa411..dd961186494 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -51,7 +51,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } -void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, const StoragePostgreSQLReplica * storage) +void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StoragePostgreSQLReplica * storage) { storages[table_name] = storage; } @@ -81,6 +81,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() } catch (...) { + /// TODO: throw tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -211,26 +212,24 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) { LOG_DEBUG(log, "Creating transaction snapshot"); - for (const auto & [table_name, storage] : storages) + for (const auto & storage_data : storages) { - storage->createNestedIfNeeded(); - auto nested_storage = storage->tryGetNested(); - - if (!nested_storage) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to create nested storage"); - try { - auto stx = std::make_unique(*connection->conn()); + auto tx = std::make_shared(*connection->conn()); /// Specific isolation level is required to read from snapshot. - stx->set_variable("transaction_isolation", "'repeatable read'"); + tx->set_variable("transaction_isolation", "'repeatable read'"); std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); - stx->exec(query_str); + tx->exec(query_str); + + storage_data.second->createNestedIfNeeded([&]() { return fetchTableStructure(tx, storage_data.first); }); + auto nested_storage = storage_data.second->getNested(); /// Load from snapshot, which will show table state before creation of replication slot. - query_str = fmt::format("SELECT * FROM {}", table_name); + /// Already connected to needed database, no need to add it to query. + query_str = fmt::format("SELECT * FROM {}", storage_data.first); Context insert_context(*context); insert_context.makeQueryContext(); @@ -245,16 +244,17 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - PostgreSQLBlockInputStream input(std::move(stx), query_str, sample_block, DEFAULT_BLOCK_SIZE); + PostgreSQLBlockInputStream input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); copyData(input, *block_io.out); - storage->setNestedLoaded(); - nested_storages[table_name] = nested_storage; + storage_data.second->setNestedLoaded(); + nested_storages[storage_data.first] = nested_storage; } catch (Exception & e) { - e.addMessage("while initial data synchronization"); + tryLogCurrentException(__PRETTY_FUNCTION__); + e.addMessage("while initial data synchronization for table {}", storage_data.first); throw; } } @@ -369,4 +369,12 @@ std::unordered_set PostgreSQLReplicationHandler::fetchTablesFromPub return tables; } + +PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure( + std::shared_ptr tx, const std::string & table_name) +{ + auto use_nulls = context->getSettingsRef().external_table_functions_use_nulls; + return fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true); +} + } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index ba6014aed1f..5a44215a612 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -3,6 +3,7 @@ #include "PostgreSQLConnection.h" #include "PostgreSQLReplicaConsumer.h" #include "PostgreSQLReplicaMetadata.h" +#include namespace DB @@ -29,10 +30,12 @@ public: void shutdownFinal(); - void addStorage(const std::string & table_name, const StoragePostgreSQLReplica * storage); + void addStorage(const std::string & table_name, StoragePostgreSQLReplica * storage); std::unordered_set fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_); + PostgreSQLTableStructure fetchTableStructure(std::shared_ptr tx, const std::string & table_name); + private: using NontransactionPtr = std::shared_ptr; @@ -65,7 +68,7 @@ private: BackgroundSchedulePool::TaskHolder startup_task; std::atomic tables_loaded = false; - std::unordered_map storages; + std::unordered_map storages; std::unordered_map nested_storages; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 490da9ce322..2eb156ad9f9 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -45,6 +45,8 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( , remote_table_name(remote_table_name_) , global_context(std::make_shared(context_.getGlobalContext())) , replication_settings(std::move(replication_settings_)) + , is_postgresql_replica_database( + DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "PostgreSQLReplica") { setInMemoryMetadata(storage_metadata); @@ -69,20 +71,6 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( } -StoragePostgreSQLReplica::StoragePostgreSQLReplica( - const StorageID & table_id_, - const String & /* metadata_path_ */, - const StorageInMemoryMetadata & storage_metadata, - const Context & context_) - : IStorage(table_id_) - , global_context(std::make_shared(context_)) -{ - setInMemoryMetadata(storage_metadata); - is_postgresql_replica_database = DatabaseCatalog::instance().getDatabase( - getStorageID().database_name)->getEngineName() == "PostgreSQLReplica"; -} - - StoragePostgreSQLReplica::StoragePostgreSQLReplica( const StorageID & table_id_, StoragePtr nested_storage_, @@ -90,9 +78,9 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( : IStorage(table_id_) , global_context(std::make_shared(context_)) , nested_storage(nested_storage_) + , is_postgresql_replica_database( + DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "PostgreSQLReplica") { - is_postgresql_replica_database = DatabaseCatalog::instance().getDatabase( - getStorageID().database_name)->getEngineName() == "PostgreSQLReplica"; } @@ -140,18 +128,70 @@ ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_t } -std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage() const +/// For single storage PostgreSQLReplica get columns and primary key columns from storage definition. +/// For database engine PostgreSQLReplica get columns and primary key columns by fetching from PostgreSQL, also using the same +/// transaction with snapshot, which is used for initial tables dump. +ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function & fetch_table_structure) { + auto create_table_query = std::make_shared(); + + auto table_id = getStorageID(); + create_table_query->table = getNestedTableName(); + create_table_query->database = table_id.database_name; + auto columns_declare_list = std::make_shared(); - auto columns_expression_list = std::make_shared(); - auto metadata_snapshot = getInMemoryMetadataPtr(); + auto order_by_expression = std::make_shared(); - for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) + auto metadata_snapshot = getInMemoryMetadataPtr(); + const auto & columns = metadata_snapshot->getColumns(); + NamesAndTypesList ordinary_columns_and_types; + + if (!columns.empty()) + { + ordinary_columns_and_types = columns.getOrdinary(); + } + else + { + auto table_structure = fetch_table_structure(); + + if (!table_structure.columns) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No columns returned for table {}.{}", table_id.database_name, table_id.table_name); + } + + StorageInMemoryMetadata storage_metadata; + + ordinary_columns_and_types = *table_structure.columns; + storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); + setInMemoryMetadata(storage_metadata); + + if (!table_structure.primary_key_columns) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No primary key columns returned for table {}.{}", table_id.database_name, table_id.table_name); + } + + auto primary_key_columns = *table_structure.primary_key_columns; + + order_by_expression->name = "tuple"; + order_by_expression->arguments = std::make_shared(); + + for (const auto & column : primary_key_columns) + { + LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "kssenii table columns {}", column.name); + order_by_expression->arguments->children.emplace_back(std::make_shared(column.name)); + } + } + + for (const auto & [name, type] : ordinary_columns_and_types) { const auto & column_declaration = std::make_shared(); - column_declaration->name = column_type_and_name.name; - column_declaration->type = getColumnDeclaration(column_type_and_name.type); + + column_declaration->name = name; + column_declaration->type = getColumnDeclaration(type); + columns_expression_list->children.emplace_back(column_declaration); } @@ -160,29 +200,18 @@ std::shared_ptr StoragePostgreSQLReplica::getColumnsListFromStorage( columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", UInt64(1))); columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", UInt64(1))); - return columns_declare_list; -} + create_table_query->set(create_table_query->columns_list, columns_declare_list); - -ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery() const -{ - auto create_table_query = std::make_shared(); - - auto table_id = getStorageID(); - create_table_query->table = getNestedTableName(); - create_table_query->database = table_id.database_name; - //create_table_query->if_not_exists = true; - - create_table_query->set(create_table_query->columns_list, getColumnsListFromStorage()); + /// Not nullptr for single storage (because throws exception if not specified), nullptr otherwise. + auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST(); auto storage = std::make_shared(); storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared("_version"))); - auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST(); - if (!primary_key_ast) - primary_key_ast = std::make_shared("key"); - - storage->set(storage->order_by, primary_key_ast); + if (primary_key_ast) + storage->set(storage->order_by, primary_key_ast); + else + storage->set(storage->order_by, order_by_expression); create_table_query->set(create_table_query->storage, storage); @@ -190,7 +219,7 @@ ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery() const } -void StoragePostgreSQLReplica::createNestedIfNeeded() const +void StoragePostgreSQLReplica::createNestedIfNeeded(const std::function & fetch_table_structure) { nested_storage = tryGetNested(); @@ -198,17 +227,10 @@ void StoragePostgreSQLReplica::createNestedIfNeeded() const return; Context context_copy(*global_context); - const auto ast_create = getCreateNestedTableQuery(); + const auto ast_create = getCreateNestedTableQuery(fetch_table_structure); InterpreterCreateQuery interpreter(ast_create, context_copy); - try - { - interpreter.execute(); - } - catch (...) - { - throw; - } + interpreter.execute(); nested_storage = getNested(); } @@ -224,7 +246,7 @@ Context StoragePostgreSQLReplica::makeGetNestedTableContext() const } -StoragePtr StoragePostgreSQLReplica::getNested() const +StoragePtr StoragePostgreSQLReplica::getNested() { if (nested_storage) return nested_storage; @@ -237,7 +259,7 @@ StoragePtr StoragePostgreSQLReplica::getNested() const } -StoragePtr StoragePostgreSQLReplica::tryGetNested() const +StoragePtr StoragePostgreSQLReplica::tryGetNested() { if (nested_storage) return nested_storage; @@ -442,7 +464,10 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) factory.registerStorage( "PostgreSQLReplica", creator_fn, - StorageFactory::StorageFeatures{ .supports_settings = true, .supports_sort_order = true, .source_access_type = AccessType::POSTGRES, + StorageFactory::StorageFeatures{ + .supports_settings = true, + .supports_sort_order = true, + .source_access_type = AccessType::POSTGRES, }); } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 277d6fc9313..e8de30afeb2 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -26,12 +26,6 @@ class StoragePostgreSQLReplica final : public ext::shared_ptr_helper; public: - StoragePostgreSQLReplica( - const StorageID & table_id_, - const String & metadata_path_, - const StorageInMemoryMetadata & storage_metadata, - const Context & context_); - StoragePostgreSQLReplica( const StorageID & table_id_, StoragePtr nested_storage_, @@ -56,15 +50,16 @@ public: /// Called right after shutdown() in case of drop query void shutdownFinal(); - void createNestedIfNeeded() const; + void createNestedIfNeeded(const std::function & fetch_table_structure); /// Can be nullptr - StoragePtr tryGetNested() const; + StoragePtr tryGetNested(); /// Throw if impossible to get - StoragePtr getNested() const; + StoragePtr getNested(); - void setNestedLoaded() const { nested_loaded.store(true); } + void setNestedLoaded() { nested_loaded.store(true); } + bool isNestedLoaded() { return nested_loaded.load(); } protected: StoragePostgreSQLReplica( @@ -80,11 +75,9 @@ private: std::shared_ptr getMaterializedColumnsDeclaration( const String name, const String type, UInt64 default_value) const; - std::shared_ptr getColumnsListFromStorage() const; - ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; - ASTPtr getCreateNestedTableQuery() const; + ASTPtr getCreateNestedTableQuery(const std::function & fetch_table_structure); std::string getNestedTableName() const; @@ -98,10 +91,10 @@ private: std::unique_ptr replication_settings; std::unique_ptr replication_handler; - bool is_postgresql_replica_database = false; + std::atomic nested_loaded = false; + StoragePtr nested_storage; - mutable std::atomic nested_loaded = false; - mutable StoragePtr nested_storage; + bool is_postgresql_replica_database = false; }; } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 08affa518e7..5312448b98b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -1,10 +1,11 @@ #include "StoragePostgreSQL.h" #if USE_LIBPQXX +#include +#include #include #include -#include #include #include #include @@ -17,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -83,9 +83,9 @@ Pipe StoragePostgreSQL::read( sample_block.insert({ column_data.type, column_data.name }); } - auto tx = std::make_unique(*connection->conn()); + auto tx = std::make_shared(*connection->conn()); return Pipe(std::make_shared( - std::make_shared(std::move(tx), query, sample_block, max_block_size_))); + std::make_shared>(tx, query, sample_block, max_block_size_))); } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index eefdff1fa87..2e3f1f6385c 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -1,6 +1,9 @@ #include #if USE_LIBPQXX +#include +#include + #include #include #include @@ -9,8 +12,6 @@ #include #include #include "registerTableFunctions.h" -#include -#include namespace DB @@ -39,7 +40,7 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(const Context & context) const { const bool use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - auto columns = fetchPostgreSQLTableStructure(connection->conn(), remote_table_name, use_nulls); + auto columns = fetchPostgreSQLTableStructure(connection->conn(), remote_table_name, use_nulls).columns; return ColumnsDescription{*columns}; } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 7d5c36a83c9..aa11b9419b4 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -13,7 +13,7 @@ instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml' postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( - key Integer NOT NULL, value Integer) + key Integer NOT NULL, value Integer, PRIMARY KEY(key)) """ def get_postgres_conn(database=False): @@ -66,7 +66,7 @@ def started_cluster(): @pytest.fixture(autouse=True) -def rabbitmq_setup_teardown(): +def postgresql_setup_teardown(): yield # run test instance.query('DROP TABLE IF EXISTS test.postgresql_replica') @@ -107,7 +107,7 @@ def test_replicating_dml(started_cluster): "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") for i in range(NUM_TABLES): - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50, 50)".format(i, i)) + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format(i, i)) for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); From 010a640ed8f92bf42ee110d0e42a2ecd38406718 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 12 Feb 2021 10:05:13 +0000 Subject: [PATCH 016/105] Fix and test different data types --- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 32 ++++++- .../test.py | 85 ++++++++++++++++++- .../test_storage_postgresql_replica/test.py | 6 +- 3 files changed, 111 insertions(+), 12 deletions(-) diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 2eb156ad9f9..7def1a317be 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -124,6 +125,33 @@ ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_t if (which.isArray()) return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + /// getName() for decimal returns 'Decimal(precison, scale)', will get an error with it + if (which.isDecimal()) + { + auto make_decimal_expression = [&](std::string type_name) + { + auto ast_expression = std::make_shared(); + + ast_expression->name = type_name; + ast_expression->arguments = std::make_shared(); + ast_expression->arguments->children.emplace_back(std::make_shared(getDecimalScale(*data_type))); + + return ast_expression; + }; + + if (which.isDecimal32()) + return make_decimal_expression("Decimal32"); + + if (which.isDecimal64()) + return make_decimal_expression("Decimal64"); + + if (which.isDecimal128()) + return make_decimal_expression("Decimal128"); + + if (which.isDecimal256()) + return make_decimal_expression("Decimal256"); + } + return std::make_shared(data_type->getName()); } @@ -178,11 +206,9 @@ ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function

name = "tuple"; order_by_expression->arguments = std::make_shared(); + //TODO: check for nullable for (const auto & column : primary_key_columns) - { - LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "kssenii table columns {}", column.name); order_by_expression->arguments->children.emplace_back(std::make_shared(column.name)); - } } for (const auto & [name, type] : ordinary_columns_and_types) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index aa11b9419b4..a9e4fd9ee30 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -36,13 +36,13 @@ def create_postgres_table(cursor, table_name): cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) -def check_tables_are_synchronized(table_name): - expected = instance.query('select * from postgres_database.{} order by key;'.format(table_name)) - result = instance.query('select * from test_database.{} order by key;'.format(table_name)) +def check_tables_are_synchronized(table_name, order_by='key'): + expected = instance.query('select * from postgres_database.{} order by {};'.format(table_name, order_by)) + result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) while result != expected: time.sleep(0.5) - result = instance.query('select * from test_database.{} order by key;'.format(table_name)) + result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) assert(result == expected) @@ -121,6 +121,8 @@ def test_replicating_dml(started_cluster): for i in range(NUM_TABLES): cursor.execute('DELETE FROM postgresql_replica_{} WHERE (value*value + {}) % 2 = 0;'.format(i, i)) + 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)) for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); @@ -132,6 +134,81 @@ def test_replicating_dml(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') +def test_different_data_types(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + cursor.execute('drop table if exists test_data_types;') + cursor.execute('drop table if exists test_array_data_type;') + + cursor.execute( + '''CREATE TABLE test_data_types ( + id integer PRIMARY KEY, a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, + h timestamp, i date, j decimal(5, 5), k numeric(5, 5))''') + + cursor.execute( + '''CREATE TABLE test_array_data_type + ( + key Integer NOT NULL PRIMARY KEY, + a Date[] NOT NULL, -- Date + b Timestamp[] NOT NULL, -- DateTime + c real[][] NOT NULL, -- Float32 + d double precision[][] NOT NULL, -- Float64 + e decimal(5, 5)[][][] NOT NULL, -- Decimal32 + f integer[][][] NOT NULL, -- Int32 + g Text[][][][][] NOT NULL, -- String + h Integer[][][], -- Nullable(Int32) + i Char(2)[][][][], -- Nullable(String) + k Char(2)[] -- Nullable(String) + )''') + + instance.query( + "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + + for i in range(10): + instance.query(''' + INSERT INTO postgres_database.test_data_types VALUES + ({}, -32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.2, 0.2)'''.format(i)) + + check_tables_are_synchronized('test_data_types', 'id'); + result = instance.query('SELECT * FROM test_database.test_data_types ORDER BY id LIMIT 1;') + assert(result == '0\t-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n') + cursor.execute('drop table test_data_types;') + + instance.query("INSERT INTO postgres_database.test_array_data_type " + "VALUES (" + "0, " + "['2000-05-12', '2000-05-12'], " + "['2000-05-12 12:12:12', '2000-05-12 12:12:12'], " + "[[1.12345], [1.12345], [1.12345]], " + "[[1.1234567891], [1.1234567891], [1.1234567891]], " + "[[[0.11111, 0.11111]], [[0.22222, 0.22222]], [[0.33333, 0.33333]]], " + "[[[1, 1], [1, 1]], [[3, 3], [3, 3]], [[4, 4], [5, 5]]], " + "[[[[['winx', 'winx', 'winx']]]]], " + "[[[1, NULL], [NULL, 1]], [[NULL, NULL], [NULL, NULL]], [[4, 4], [5, 5]]], " + "[[[[NULL]]]], " + "[]" + ")") + + expected = ( + "0\t" + + "['2000-05-12','2000-05-12']\t" + + "['2000-05-12 12:12:12','2000-05-12 12:12:12']\t" + + "[[1.12345],[1.12345],[1.12345]]\t" + + "[[1.1234567891],[1.1234567891],[1.1234567891]]\t" + + "[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t" + "[[[1,1],[1,1]],[[3,3],[3,3]],[[4,4],[5,5]]]\t" + "[[[[['winx','winx','winx']]]]]\t" + "[[[1,NULL],[NULL,1]],[[NULL,NULL],[NULL,NULL]],[[4,4],[5,5]]]\t" + "[[[[NULL]]]]\t" + "[]\n" + ) + + check_tables_are_synchronized('test_array_data_type'); + result = instance.query('SELECT * FROM test_database.test_array_data_type ORDER BY key;') + instance.query("DROP DATABASE test_database") + assert(result == expected) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 8773c484039..646364a7ca3 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -52,7 +52,6 @@ def started_cluster(): instance.query(''' CREATE DATABASE postgres_database ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')''') - instance.query('CREATE DATABASE test') yield cluster @@ -140,6 +139,7 @@ def test_detach_attach_is_ok(started_cluster): instance.query('DETACH TABLE test.postgresql_replica') instance.query('ATTACH TABLE test.postgresql_replica') + time.sleep(0.5) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) @@ -206,10 +206,6 @@ def test_replicating_delete_queries(started_cluster): time.sleep(0.2) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') - postgresql_replica_check_result(result, True) - - result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)") result = instance.query('SELECT count() FROM test.postgresql_replica;') From 219dece1d0c732a941f0b1ec1b8cfed9b3d1c276 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 12 Feb 2021 15:48:01 +0000 Subject: [PATCH 017/105] Slightly better --- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 30 +++-- .../PostgreSQL/DatabasePostgreSQLReplica.h | 5 +- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 104 +++++++++--------- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 1 - .../PostgreSQLReplicationHandler.cpp | 31 +++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 3 + .../PostgreSQL/StoragePostgreSQLReplica.cpp | 24 ++-- .../PostgreSQL/StoragePostgreSQLReplica.h | 5 +- 8 files changed, 99 insertions(+), 104 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index a9b07a0c65b..049c17eaf8a 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -55,6 +55,7 @@ DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( : DatabaseOrdinary( database_name_, metadata_path_, "data/" + escapeForFileName(database_name_) + "/", "DatabasePostgreSQLReplica (" + database_name_ + ")", context) + , log(&Poco::Logger::get("PostgreSQLReplicaDatabaseEngine")) , global_context(context.getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) @@ -117,7 +118,7 @@ void DatabasePostgreSQLReplica::startSynchronization() } } - LOG_TRACE(&Poco::Logger::get("PostgreSQLReplicaDatabaseEngine"), "Loaded {} tables. Starting synchronization", tables.size()); + LOG_TRACE(log, "Loaded {} tables. Starting synchronization", tables.size()); replication_handler->startup(); } @@ -173,11 +174,20 @@ StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, con } -/// TODO: assert called from sync thread template void DatabasePostgreSQLReplica::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) { - Base::createTable(context, name, table, query); + if (context.hasQueryContext()) + { + auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; + if (storage_set.find("ReplacingMergeTree") != storage_set.end()) + { + Base::createTable(context, name, table, query); + return; + } + } + + LOG_WARNING(log, "Create table query allowed only for ReplacingMergeTree engine and from synchronization thread"); } @@ -188,20 +198,6 @@ void DatabasePostgreSQLReplica::dropTable(const Context & context, const S } -template -void DatabasePostgreSQLReplica::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) -{ - Base::attachTable(name, table, relative_table_path); -} - - -template -StoragePtr DatabasePostgreSQLReplica::detachTable(const String & name) -{ - return Base::detachTable(name); -} - - template void DatabasePostgreSQLReplica::drop(const Context & context) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h index a73acd7b27b..5847f47ebef 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h @@ -53,10 +53,6 @@ public: void dropTable(const Context & context, const String & name, bool no_delay) override; - void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; - - StoragePtr detachTable(const String & name) override; - void drop(const Context & context) override; void shutdown() override; @@ -66,6 +62,7 @@ private: void startSynchronization(); StoragePtr getStorage(const String & name); + Poco::Logger * log; const Context global_context; String metadata_path; ASTPtr database_engine_define; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 3435abc1fa9..50896fa8394 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -1,4 +1,5 @@ #include "PostgreSQLReplicaConsumer.h" +#include "StoragePostgreSQLReplica.h" #include #include @@ -85,7 +86,6 @@ void PostgreSQLReplicaConsumer::stopSynchronization() void PostgreSQLReplicaConsumer::synchronizationStream() { auto start_time = std::chrono::steady_clock::now(); - LOG_TRACE(log, "Starting synchronization stream"); while (!stop_synchronization) { @@ -105,7 +105,6 @@ void PostgreSQLReplicaConsumer::synchronizationStream() void PostgreSQLReplicaConsumer::insertValue(BufferData & buffer, const std::string & value, size_t column_idx) { - LOG_TRACE(log, "INSERTING VALUE {}", value); const auto & sample = buffer.description.sample_block.getByPosition(column_idx); bool is_nullable = buffer.description.types[column_idx].second; @@ -198,21 +197,24 @@ void PostgreSQLReplicaConsumer::readTupleData( BufferData & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos, size); - /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data - LOG_DEBUG(log, "num_columns {}", num_columns); + LOG_DEBUG(log, "number of columns {}", num_columns); + for (int column_idx = 0; column_idx < num_columns; ++column_idx) { + /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data char identifier = readInt8(message, pos, size); Int32 col_len = readInt32(message, pos, size); String value; + for (int i = 0; i < col_len; ++i) { value += readInt8(message, pos, size); } + /// TODO: Check for null values and use insertDefaultValue insertValue(buffer, value, column_idx); - LOG_DEBUG(log, "identifier {}, col_len {}, value {}", identifier, col_len, value); + LOG_DEBUG(log, "Identifier: {}, column length: {}, value: {}", identifier, col_len, value); } switch (type) @@ -233,6 +235,9 @@ void PostgreSQLReplicaConsumer::readTupleData( } case PostgreSQLQuery::UPDATE: { + /// TODO: If table has primary key, skip old value and remove first insert with -1. + // Otherwise use replica identity full (with check) and use fisrt insert. + if (old_value) buffer.columns[num_columns]->insert(Int8(-1)); else @@ -245,33 +250,31 @@ void PostgreSQLReplicaConsumer::readTupleData( } } -/// test relation id can be shuffled ? + +/// https://www.postgresql.org/docs/13/protocol-logicalrep-message-formats.html void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replication_message, size_t size) { /// Skip '\x' size_t pos = 2; char type = readInt8(replication_message, pos, size); - LOG_TRACE(log, "TYPE: {}", type); + LOG_DEBUG(log, "Type of replication message: {}", type); + switch (type) { case 'B': // Begin { - Int64 transaction_end_lsn = readInt64(replication_message, pos, size); - Int64 transaction_commit_timestamp = readInt64(replication_message, pos, size); - LOG_DEBUG(log, "transaction lsn {}, transaction commit timespamp {}", - transaction_end_lsn, transaction_commit_timestamp); + readInt64(replication_message, pos, size); /// Int64 transaction end lsn + readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp break; } case 'C': // Commit { - readInt8(replication_message, pos, size); - Int64 commit_lsn = readInt64(replication_message, pos, size); - Int64 transaction_end_lsn = readInt64(replication_message, pos, size); - /// Since postgres epoch - Int64 transaction_commit_timestamp = readInt64(replication_message, pos, size); - LOG_DEBUG(log, "commit lsn {}, transaction lsn {}, transaction commit timestamp {}", - commit_lsn, transaction_end_lsn, transaction_commit_timestamp); + readInt8(replication_message, pos, size); /// unused flags + readInt64(replication_message, pos, size); /// Int64 commit lsn + readInt64(replication_message, pos, size); /// Int64 transaction end lsn + readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp + final_lsn = current_lsn; break; } @@ -280,38 +283,49 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati case 'R': // Relation { Int32 relation_id = readInt32(replication_message, pos, size); - String relation_namespace, relation_name; - readString(replication_message, pos, size, relation_namespace); + String relation_namespace, relation_name; + + readString(replication_message, pos, size, relation_namespace); readString(replication_message, pos, size, relation_name); + + /// TODO: Save relation id (unique to tables) and check if they might be shuffled in current block. + /// If shuffled, store tables based on those id's and insert accordingly. table_to_insert = relation_name; tables_to_sync.insert(table_to_insert); - LOG_DEBUG(log, "INSERTING TABLE {}", table_to_insert); Int8 replica_identity = readInt8(replication_message, pos, size); Int16 num_columns = readInt16(replication_message, pos, size); + /// TODO: If replica identity is not full, check if there will be a full columns list. LOG_DEBUG(log, - "Replication message type 'R', relation_id: {}, namespace: {}, relation name {}, replica identity {}, columns number {}", + "INFO: relation id: {}, namespace: {}, relation name: {}, replica identity: {}, columns number: {}", relation_id, relation_namespace, relation_name, replica_identity, num_columns); Int8 key; Int32 data_type_id, type_modifier; + + /// TODO: Check here if table structure has changed and, if possible, change table structure and redump table. for (uint16_t i = 0; i < num_columns; ++i) { String column_name; key = readInt8(replication_message, pos, size); readString(replication_message, pos, size, column_name); + data_type_id = readInt32(replication_message, pos, size); type_modifier = readInt32(replication_message, pos, size); - LOG_DEBUG(log, "Key {}, column name {}, data type id {}, type modifier {}", key, column_name, data_type_id, type_modifier); + + LOG_DEBUG(log, + "Key: {}, column name: {}, data type id: {}, type modifier: {}", + key, column_name, data_type_id, type_modifier); } if (storages.find(table_to_insert) == storages.end()) { - throw Exception(ErrorCodes::UNKNOWN_TABLE, - "Table {} does not exist, but is included in replication stream", table_to_insert); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Storage for table {} does not exist, but is included in replication stream", table_to_insert); } + [[maybe_unused]] auto buffer_iter = buffers.find(table_to_insert); assert(buffer_iter != buffers.end()); @@ -324,7 +338,8 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati Int32 relation_id = readInt32(replication_message, pos, size); Int8 new_tuple = readInt8(replication_message, pos, size); - LOG_DEBUG(log, "relationID {}, newTuple {} current insert tabel {}", relation_id, new_tuple, table_to_insert); + LOG_DEBUG(log, "relationID: {}, newTuple: {}, current insert table: {}", relation_id, new_tuple, table_to_insert); + auto buffer = buffers.find(table_to_insert); if (buffer == buffers.end()) { @@ -341,14 +356,17 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati LOG_DEBUG(log, "relationID {}, key {} current insert table {}", relation_id, primary_key_or_old_tuple_data, table_to_insert); + /// TODO: Two cases: based on primary keys and based on replica identity full. + /// Add first one and add a check for second one. + auto buffer = buffers.find(table_to_insert); readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); if (pos + 1 < size) { Int8 new_tuple_data = readInt8(replication_message, pos, size); - LOG_DEBUG(log, "new tuple data {}", new_tuple_data); readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE); + LOG_DEBUG(log, "new tuple data: {}", new_tuple_data); } break; @@ -356,11 +374,9 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati case 'D': // Delete { Int32 relation_id = readInt32(replication_message, pos, size); - //Int8 index_replica_identity = readInt8(replication_message, pos); Int8 full_replica_identity = readInt8(replication_message, pos, size); - LOG_DEBUG(log, "relationID {}, full replica identity {}", - relation_id, full_replica_identity); + LOG_DEBUG(log, "relationID: {}, full replica identity: {}", relation_id, full_replica_identity); auto buffer = buffers.find(table_to_insert); readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE); @@ -377,38 +393,32 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr tx) { - LOG_TRACE(log, "AVAILABLE TABLES {}", tables_to_sync.size()); for (const auto & table_name : tables_to_sync) { try { - LOG_TRACE(log, "ATTEMPT SYNCING TABLE {}", table_name); auto & buffer = buffers.find(table_name)->second; Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns)); if (result_rows.rows()) { - LOG_TRACE(log, "SYNCING TABLE {} rows {} max_block_size {}", table_name, result_rows.rows(), max_block_size); - metadata.commitMetadata(final_lsn, [&]() { - Context insert_context(*context); + auto storage = storages[table_name]; + + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + + auto insert_context(*context); insert_context.makeQueryContext(); insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - auto insert = std::make_shared(); - insert->table_id = storages[table_name]->getStorageID(); - InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); - - /// TODO: what if one block is not enough OneBlockInputStream input(result_rows); copyData(input, *block_io.out); - LOG_TRACE(log, "TABLE SYNC END"); - auto actual_lsn = advanceLSN(tx); buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); @@ -422,6 +432,7 @@ void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr } } + LOG_DEBUG(log, "Table sync end for {} tables", tables_to_sync.size()); tables_to_sync.clear(); tx->commit(); } @@ -429,8 +440,6 @@ void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr tx) { - LOG_TRACE(log, "CURRENT LSN FROM TO {}", final_lsn); - std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn); pqxx::result result{tx->exec(query_str)}; @@ -468,7 +477,6 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() if (!row) { - LOG_TRACE(log, "STREAM REPLICATION END"); stream.complete(); if (slot_empty) @@ -481,19 +489,17 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() } slot_empty = false; - current_lsn = (*row)[0]; - LOG_TRACE(log, "Replication message: {}", (*row)[1]); + processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } } catch (const pqxx::sql_error & e) { - /// sql replication interface has the problem that it registers relcache + /// For now sql replication interface is used and it has the problem that it registers relcache /// callbacks on each pg_logical_slot_get_changes and there is no way to invalidate them: /// https://github.com/postgres/postgres/blob/master/src/backend/replication/pgoutput/pgoutput.c#L1128 /// So at some point will get out of limit and then they will be cleaned. - std::string error_message = e.what(); if (error_message.find("out of relcache_callback_list slots") == std::string::npos) tryLogCurrentException(__PRETTY_FUNCTION__); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index 9cd5f368a6d..a5ae25c3f53 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -27,7 +27,6 @@ PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadat void PostgreSQLReplicaMetadata::readMetadata() { - LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaMetadata"), "kssenii 1 {}", metadata_file); if (Poco::File(metadata_file).exists()) { ReadBufferFromFile in(metadata_file, DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index dd961186494..5a1ab778382 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes static const auto reschedule_ms = 500; -/// TODO: context should be const +/// TODO: add test for syncing only subset of databse tables PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, @@ -81,7 +81,6 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() } catch (...) { - /// TODO: throw tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -101,7 +100,6 @@ bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr() == "t"); - /// TODO: check if publication is still valid? if (publication_exists) LOG_TRACE(log, "Publication {} already exists. Using existing version", publication_name); @@ -131,26 +129,26 @@ void PostgreSQLReplicationHandler::createPublication(std::shared_ptr e.addMessage("while creating pg_publication"); throw; } - - /// TODO: check replica identity? - /// Requires changed replica identity for included table to be able to receive old values of updated rows. } void PostgreSQLReplicationHandler::startSynchronization() { - /// used commands require a specific transaction isolation mode. + /// Used commands require a specific transaction isolation mode. replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); auto tx = std::make_shared(*replication_connection->conn()); + bool new_publication = false; + if (publication_name.empty()) { publication_name = fmt::format("{}_ch_publication", database_name); - /// Publication defines what tables are included into replication stream. Should be deleted only if MaterializePostgreSQL - /// table is dropped. if (!isPublicationExist(tx)) + { createPublication(tx); + new_publication = true; + } } else if (!isPublicationExist(tx)) { @@ -175,9 +173,11 @@ void PostgreSQLReplicationHandler::startSynchronization() { initial_sync(); } - else if (!Poco::File(metadata_path).exists()) + else if (!Poco::File(metadata_path).exists() || new_publication) { - /// If replication slot exists and metadata file (where last synced version is written) does not exist, it is not normal. + /// In case of some failure, the following cases are possible (since publication and replication slot are reused): + /// 1. If replication slot exists and metadata file (where last synced version is written) does not exist, it is not ok. + /// 2. If created a new publication and replication slot existed before it was created, it is not ok. dropReplicationSlot(ntx, replication_slot); initial_sync(); } @@ -210,8 +210,6 @@ void PostgreSQLReplicationHandler::startSynchronization() void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) { - LOG_DEBUG(log, "Creating transaction snapshot"); - for (const auto & storage_data : storages) { try @@ -231,12 +229,9 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) /// Already connected to needed database, no need to add it to query. query_str = fmt::format("SELECT * FROM {}", storage_data.first); - Context insert_context(*context); - insert_context.makeQueryContext(); - insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - auto insert = std::make_shared(); insert->table_id = nested_storage->getStorageID(); + auto insert_context = storage_data.second->makeNestedTableContext(); InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); @@ -259,7 +254,7 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) } } - LOG_DEBUG(log, "Done loading from snapshot"); + LOG_DEBUG(log, "Table dump end"); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5a44215a612..e36e1cd0490 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -40,12 +40,15 @@ private: using NontransactionPtr = std::shared_ptr; bool isPublicationExist(std::shared_ptr tx); + bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); void createPublication(std::shared_ptr tx); + void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name); void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name); + void dropPublication(NontransactionPtr ntx); void waitConnectionAndStart(); diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 7def1a317be..e45d678ba0b 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -252,23 +252,23 @@ void StoragePostgreSQLReplica::createNestedIfNeeded(const std::functiondatabase = table_id.database_name; ast_drop->if_exists = true; - auto drop_context(*global_context); - drop_context.makeQueryContext(); - - auto interpreter = InterpreterDropQuery(ast_drop, drop_context); + auto context = makeNestedTableContext(); + auto interpreter = InterpreterDropQuery(ast_drop, context); interpreter.execute(); } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index e8de30afeb2..4059cb744e6 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -58,7 +58,10 @@ public: /// Throw if impossible to get StoragePtr getNested(); + Context makeNestedTableContext() const; + void setNestedLoaded() { nested_loaded.store(true); } + bool isNestedLoaded() { return nested_loaded.load(); } protected: @@ -81,8 +84,6 @@ private: std::string getNestedTableName() const; - Context makeGetNestedTableContext() const; - void dropNested(); std::string remote_table_name; From 44f4f1a41208bc1b65062b21301ca95134b591cf Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 12 Feb 2021 18:21:55 +0000 Subject: [PATCH 018/105] Allow to replicate a subset of database tables --- src/Databases/DatabaseFactory.cpp | 5 +- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 10 +- .../PostgreSQL/DatabasePostgreSQLReplica.h | 8 +- .../PostgreSQL/PostgreSQLReplicaSettings.h | 3 +- .../PostgreSQLReplicationHandler.cpp | 146 ++++++++---------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 12 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 7 +- .../test.py | 54 ++++++- .../test_storage_postgresql_replica/test.py | 1 + 9 files changed, 140 insertions(+), 106 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index d4b7674f73b..f92c0157e74 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -103,8 +103,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String && engine_name != "Lazy" && engine_define->engine->arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); - if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || - engine_define->sample_by || (!endsWith(engine_name, "MySQL") && engine_define->settings)) + if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || + engine_define->order_by || engine_define->sample_by || + (!endsWith(engine_name, "MySQL") && (engine_name != "PostgreSQLReplica") && engine_define->settings)) throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 049c17eaf8a..0052f8fe10d 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -39,8 +39,6 @@ namespace ErrorCodes static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; -/// TODO: add detach, after which table structure is updated, need to update StoragePtr and recreate nested_storage. -/// Also pass new storagePtr to replication Handler. Stop replication stream mean while? template<> DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( @@ -91,19 +89,15 @@ DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( template void DatabasePostgreSQLReplica::startSynchronization() { - auto publication_name = global_context.getMacros()->expand(settings->postgresql_publication_name.value); - auto replication_slot = global_context.getMacros()->expand(settings->postgresql_replication_slot_name.value); - replication_handler = std::make_unique( remote_database_name, connection->conn_str(), metadata_path + METADATA_SUFFIX, std::make_shared(global_context), - replication_slot, - publication_name, settings->postgresql_max_block_size.changed ? settings->postgresql_max_block_size.value - : (global_context.getSettingsRef().max_insert_block_size.value)); + : (global_context.getSettingsRef().max_insert_block_size.value), + global_context.getMacros()->expand(settings->postgresql_tables_list.value)); std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->conn()); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h index 5847f47ebef..24763e697e6 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h @@ -40,6 +40,7 @@ public: std::unique_ptr settings_); String getEngineName() const override { return "PostgreSQLReplica"; } + String getMetadataPath() const override { return metadata_path; } void loadStoredObjects(Context &, bool, bool force_attach) override; @@ -60,6 +61,7 @@ public: private: void startSynchronization(); + StoragePtr getStorage(const String & name); Poco::Logger * log; @@ -72,12 +74,6 @@ private: std::shared_ptr replication_handler; std::map tables; - - bool checkPostgresTable(const String & table_name) const; - std::unordered_set fetchTablesList() const; - StoragePtr fetchTable(const String & table_name, const Context & context, const bool table_checked) const; - void removeOutdatedTables(); - ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h index 72b7f98ea6e..1dbd6b0a65b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h @@ -8,9 +8,8 @@ namespace DB #define LIST_OF_POSTGRESQL_REPLICA_SETTINGS(M) \ - M(String, postgresql_replication_slot_name, "", "PostgreSQL replication slot name.", 0) \ - M(String, postgresql_publication_name, "", "PostgreSQL publication name.", 0) \ M(UInt64, postgresql_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ + M(String, postgresql_tables_list, "", "List of tables for PostgreSQLReplica database engine", 0) \ DECLARE_SETTINGS_TRAITS(PostgreSQLReplicaSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 5a1ab778382..76dede3ff4c 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -29,22 +29,19 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & conn_str, const std::string & metadata_path_, std::shared_ptr context_, - const std::string & publication_name_, - const std::string & replication_slot_name_, - const size_t max_block_size_) + const size_t max_block_size_, + const String tables_list_) : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) , context(context_) , database_name(database_name_) , connection_str(conn_str) , metadata_path(metadata_path_) - , publication_name(publication_name_) - , replication_slot(replication_slot_name_) , max_block_size(max_block_size_) + , tables_list(tables_list_) , connection(std::make_shared(conn_str)) - , replication_connection(std::make_shared(fmt::format("{} replication=database", connection->conn_str()))) { - if (replication_slot.empty()) - replication_slot = fmt::format("{}_ch_replication_slot", database_name); + replication_slot = fmt::format("{}_ch_replication_slot", database_name); + publication_name = fmt::format("{}_ch_publication", database_name); startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); startup_task->deactivate(); @@ -93,71 +90,12 @@ void PostgreSQLReplicationHandler::shutdown() } -bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr tx) -{ - std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); - pqxx::result result{tx->exec(query_str)}; - assert(!result.empty()); - bool publication_exists = (result[0][0].as() == "t"); - - if (publication_exists) - LOG_TRACE(log, "Publication {} already exists. Using existing version", publication_name); - - return publication_exists; -} - - -void PostgreSQLReplicationHandler::createPublication(std::shared_ptr tx) -{ - String table_names; - for (const auto & storage_data : storages) - { - if (!table_names.empty()) - table_names += ", "; - table_names += storage_data.first; - } - - /// 'ONLY' means just a table, without descendants. - std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, table_names); - try - { - tx->exec(query_str); - LOG_TRACE(log, "Created publication {} with tables list: {}", publication_name, table_names); - } - catch (Exception & e) - { - e.addMessage("while creating pg_publication"); - throw; - } -} - - void PostgreSQLReplicationHandler::startSynchronization() { - /// Used commands require a specific transaction isolation mode. + createPublicationIfNeeded(connection->conn()); + + auto replication_connection = std::make_shared(fmt::format("{} replication=database", connection->conn_str())); replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); - - auto tx = std::make_shared(*replication_connection->conn()); - bool new_publication = false; - - if (publication_name.empty()) - { - publication_name = fmt::format("{}_ch_publication", database_name); - - if (!isPublicationExist(tx)) - { - createPublication(tx); - new_publication = true; - } - } - else if (!isPublicationExist(tx)) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Publication name '{}' is spesified in table arguments, but it does not exist", publication_name); - } - tx->commit(); - auto ntx = std::make_shared(*replication_connection->conn()); std::string snapshot_name, start_lsn; @@ -173,7 +111,7 @@ void PostgreSQLReplicationHandler::startSynchronization() { initial_sync(); } - else if (!Poco::File(metadata_path).exists() || new_publication) + else if (!Poco::File(metadata_path).exists() || new_publication_created) { /// In case of some failure, the following cases are possible (since publication and replication slot are reused): /// 1. If replication slot exists and metadata file (where last synced version is written) does not exist, it is not ok. @@ -258,6 +196,59 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) } +bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr tx) +{ + std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); + pqxx::result result{tx->exec(query_str)}; + assert(!result.empty()); + bool publication_exists = (result[0][0].as() == "t"); + + if (publication_exists) + LOG_INFO(log, "Publication {} already exists. Using existing version", publication_name); + + return publication_exists; +} + + +void PostgreSQLReplicationHandler::createPublicationIfNeeded( + PostgreSQLConnection::ConnectionPtr connection_) +{ + if (new_publication_created) + return; + + auto tx = std::make_shared(*connection_); + + if (!isPublicationExist(tx)) + { + if (tables_list.empty()) + { + for (const auto & storage_data : storages) + { + if (!tables_list.empty()) + tables_list += ", "; + tables_list += storage_data.first; + } + } + + /// 'ONLY' means just a table, without descendants. + std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, tables_list); + try + { + tx->exec(query_str); + new_publication_created = true; + LOG_TRACE(log, "Created publication {} with tables list: {}", publication_name, tables_list); + } + catch (Exception & e) + { + e.addMessage("while creating pg_publication"); + throw; + } + } + + tx->commit(); +} + + bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name) { std::string query_str = fmt::format("SELECT active, restart_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); @@ -304,9 +295,6 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, st void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr ntx) { - if (publication_name.empty()) - return; - std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); ntx->exec(query_str); } @@ -328,7 +316,6 @@ void PostgreSQLReplicationHandler::shutdownFinal() } -/// TODO: publication can be created with option `whole_database`. Check this case. std::unordered_set PostgreSQLReplicationHandler::fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_) { auto publication_exist = [&]() @@ -339,14 +326,17 @@ std::unordered_set PostgreSQLReplicationHandler::fetchRequiredTable return exist; }; - if (publication_name.empty() || !publication_exist()) + if (publication_exist()) + { + return fetchTablesFromPublication(connection_); + } + else if (tables_list.empty()) { - /// Replicate the whole database and create our own pg_publication return fetchPostgreSQLTablesList(connection_); } else { - /// Replicate only tables, which are included in a pg_publication + createPublicationIfNeeded(connection_); return fetchTablesFromPublication(connection_); } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index e36e1cd0490..a51c497c21d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -20,9 +20,8 @@ public: const std::string & conn_str_, const std::string & metadata_path_, std::shared_ptr context_, - const std::string & publication_slot_name_, - const std::string & replication_slot_name_, - const size_t max_block_size_); + const size_t max_block_size_, + const String tables_list = ""); void startup(); @@ -43,7 +42,7 @@ private: bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); - void createPublication(std::shared_ptr tx); + void createPublicationIfNeeded(PostgreSQLConnection::ConnectionPtr connection_); void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name); @@ -62,14 +61,15 @@ private: Poco::Logger * log; std::shared_ptr context; const std::string database_name, connection_str, metadata_path; - std::string publication_name, replication_slot; const size_t max_block_size; + std::string tables_list, replication_slot, publication_name; - PostgreSQLConnectionPtr connection, replication_connection; + PostgreSQLConnectionPtr connection; std::shared_ptr consumer; BackgroundSchedulePool::TaskHolder startup_task; std::atomic tables_loaded = false; + bool new_publication_created = false; std::unordered_map storages; std::unordered_map nested_storages; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index e45d678ba0b..30855e1b2a8 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -62,8 +62,6 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( connection_str, metadata_path, global_context, - global_context->getMacros()->expand(replication_settings->postgresql_replication_slot_name.value), - global_context->getMacros()->expand(replication_settings->postgresql_publication_name.value), replication_settings->postgresql_max_block_size.changed ? replication_settings->postgresql_max_block_size.value : (global_context->getSettingsRef().max_insert_block_size.value) @@ -346,7 +344,10 @@ void StoragePostgreSQLReplica::dropNested() NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const { - return NamesAndTypesList{}; + if (nested_storage) + return nested_storage->getVirtuals(); + + return {}; } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index a9e4fd9ee30..10cd5d2a263 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -32,6 +32,7 @@ def create_postgres_db(cursor, name): def create_postgres_table(cursor, table_name): + cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) cursor.execute(postgres_table_template.format(table_name)) cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) @@ -71,7 +72,8 @@ def postgresql_setup_teardown(): instance.query('DROP TABLE IF EXISTS test.postgresql_replica') -def test_load_and_sync_all_database(started_cluster): +def test_load_and_sync_all_database_tables(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) cursor = conn.cursor() NUM_TABLES = 5 @@ -95,6 +97,7 @@ def test_load_and_sync_all_database(started_cluster): def test_replicating_dml(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) cursor = conn.cursor() NUM_TABLES = 5 @@ -135,6 +138,7 @@ def test_replicating_dml(started_cluster): def test_different_data_types(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) cursor = conn.cursor() cursor.execute('drop table if exists test_data_types;') @@ -209,6 +213,54 @@ def test_different_data_types(started_cluster): assert(result == expected) +def test_load_and_sync_subset_of_database_tables(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 10 + + publication_tables = '' + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(50)".format(i)) + + if i < NUM_TABLES/2: + if publication_tables != '': + publication_tables += ', ' + publication_tables += table_name + + instance.query(''' + CREATE DATABASE test_database + ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + SETTINGS postgresql_tables_list = '{}'; + '''.format(publication_tables)) + assert 'test_database' in instance.query('SHOW DATABASES') + + time.sleep(1) + + result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''') + assert(int(result) == NUM_TABLES/2) + + database_tables = instance.query('SHOW TABLES FROM test_database') + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + if i < NUM_TABLES/2: + assert table_name in database_tables + else: + assert table_name not in database_tables + instance.query("INSERT INTO postgres_database.{} SELECT 50 + number, {} from numbers(100)".format(table_name, i)) + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + if i < NUM_TABLES/2: + check_tables_are_synchronized(table_name); + cursor.execute('drop table {};'.format(table_name)) + + instance.query("DROP DATABASE test_database") + assert 'test_database' not in instance.query('SHOW DATABASES') + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 646364a7ca3..57d3b5288fb 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -30,6 +30,7 @@ def create_postgres_db(cursor, name): cursor.execute("CREATE DATABASE {}".format(name)) def create_postgres_table(cursor, table_name): + cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) cursor.execute(postgres_table_template.format(table_name)) cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) From 95c93aca41f39875f3080ce29d2b0fdfc9d67f66 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 13 Feb 2021 20:46:52 +0000 Subject: [PATCH 019/105] Better table sync --- .../PostgreSQLBlockInputStream.cpp | 2 -- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 11 ---------- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 6 ++++-- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 21 ++++++++++++++++--- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 6 ------ .../PostgreSQLReplicationHandler.cpp | 9 -------- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 4 +--- 7 files changed, 23 insertions(+), 36 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 9e3c6b1bb89..7e9aa40e904 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -138,8 +138,6 @@ void PostgreSQLBlockInputStream::readSuffix() } } - - } #endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 0052f8fe10d..7960270391f 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -27,19 +27,8 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int NOT_IMPLEMENTED; - extern const int UNKNOWN_TABLE; - extern const int TABLE_IS_DROPPED; - extern const int TABLE_ALREADY_EXISTS; -} - - static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; - template<> DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( const Context & context, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 50896fa8394..dd915197093 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -45,7 +45,7 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( { for (const auto & [table_name, storage] : storages) { - buffers.emplace(table_name, BufferData(storage->getInMemoryMetadata().getSampleBlock())); + buffers.emplace(table_name, BufferData(storage)); } wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ synchronizationStream(); }); @@ -408,15 +408,17 @@ void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr auto insert = std::make_shared(); insert->table_id = storage->getStorageID(); + insert->columns = buffer.columnsAST; auto insert_context(*context); insert_context.makeQueryContext(); insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - InterpreterInsertQuery interpreter(insert, insert_context); + InterpreterInsertQuery interpreter(insert, insert_context, true); auto block_io = interpreter.execute(); OneBlockInputStream input(result_rows); + assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica table sync"); copyData(input, *block_io.out); auto actual_lsn = advanceLSN(tx); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 0973ba7f785..cb1c76829f1 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -48,16 +49,30 @@ private: { ExternalResultDescription description; MutableColumns columns; + std::shared_ptr columnsAST; /// Needed for insertPostgreSQLValue() method to parse array std::unordered_map array_info; - BufferData(const Block block) + BufferData(StoragePtr storage) { - description.init(block); + const auto storage_metadata = storage->getInMemoryMetadataPtr(); + description.init(storage_metadata->getSampleBlock()); columns = description.sample_block.cloneEmptyColumns(); - for (const auto idx : ext::range(0, description.sample_block.columns())) + const auto & storage_columns = storage_metadata->getColumns().getAllPhysical(); + auto insert_columns = std::make_shared(); + size_t idx = 0; + assert(description.sample_block.columns() == storage_columns.size()); + + for (const auto & column : storage_columns) + { if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); + idx++; + + insert_columns->children.emplace_back(std::make_shared(column.name)); + } + + columnsAST = std::move(insert_columns); } }; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index a5ae25c3f53..1eb4abb6a6d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -11,12 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadata_file_path) : metadata_file(metadata_file_path) , tmp_metadata_file(metadata_file_path + ".tmp") diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 76dede3ff4c..bc50e5ab270 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -13,17 +13,8 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TABLE; - extern const int LOGICAL_ERROR; -} - static const auto reschedule_ms = 500; -/// TODO: add test for syncing only subset of databse tables - PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const std::string & conn_str, diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 30855e1b2a8..9c514595ccc 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree"; @@ -51,9 +52,6 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( { setInMemoryMetadata(storage_metadata); - is_postgresql_replica_database = DatabaseCatalog::instance().getDatabase( - getStorageID().database_name)->getEngineName() == "PostgreSQLReplica"; - auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath() + "/.metadata_" + table_id_.database_name + "_" + table_id_.table_name; From 272431bcae2f1fee8eaedc231eb1b4f1054159a2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 15 Feb 2021 22:49:13 +0000 Subject: [PATCH 020/105] Fix types check, better tests, try fix build --- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 15 ++++++-- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 2 +- .../PostgreSQL/PostgreSQLReplicaMetadata.cpp | 2 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 2 +- .../test.py | 34 ++++++++++++++----- 5 files changed, 41 insertions(+), 14 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 7960270391f..29c3ac7491a 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -131,7 +131,18 @@ void DatabasePostgreSQLReplica::loadStoredObjects( Context & context, bool has_force_restore_data_flag, bool force_attach) { Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach); - startSynchronization(); + + try + { + startSynchronization(); + } + catch (...) + { + tryLogCurrentException(Base::log, "Cannot load nested database objects for PostgreSQL database engine."); + + if (!force_attach) + throw; + } } @@ -207,7 +218,7 @@ DatabaseTablesIteratorPtr DatabasePostgreSQLReplica::getTablesIterator( Tables nested_tables; for (const auto & [table_name, storage] : tables) { - auto nested_storage = storage->as()->tryGetNested(); + auto nested_storage = storage->template as()->tryGetNested(); if (nested_storage) nested_tables[table_name] = nested_storage; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index dd915197093..c2d59ddf516 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -236,7 +236,7 @@ void PostgreSQLReplicaConsumer::readTupleData( case PostgreSQLQuery::UPDATE: { /// TODO: If table has primary key, skip old value and remove first insert with -1. - // Otherwise use replica identity full (with check) and use fisrt insert. + // Otherwise use replica identity full (with check) and use first insert. if (old_value) buffer.columns[num_columns]->insert(Int8(-1)); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index 1eb4abb6a6d..8a45b415ad0 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -66,7 +66,7 @@ void PostgreSQLReplicaMetadata::writeMetadata(bool append_metadata) } -/// While data is recieved, version is updated. Before table sync, write last version to tmp file. +/// While data is received, version is updated. Before table sync, write last version to tmp file. /// Then sync data to table and rename tmp to non-tmp. void PostgreSQLReplicaMetadata::commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc) { diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 9c514595ccc..4348330c832 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -121,7 +121,7 @@ ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_t if (which.isArray()) return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); - /// getName() for decimal returns 'Decimal(precison, scale)', will get an error with it + /// getName() for decimal returns 'Decimal(precision, scale)', will get an error with it if (which.isDecimal()) { auto make_decimal_expression = [&](std::string type_name) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 10cd5d2a263..50f7ded4354 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -37,10 +37,19 @@ def create_postgres_table(cursor, table_name): cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) +def assert_nested_table_is_created(table_name): + database_tables = instance.query('SHOW TABLES FROM test_database') + while table_name not in database_tables: + time.sleep(0.2) + database_tables = instance.query('SHOW TABLES FROM test_database') + assert(table_name in database_tables) + + def check_tables_are_synchronized(table_name, order_by='key'): expected = instance.query('select * from postgres_database.{} order by {};'.format(table_name, order_by)) result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) + assert_nested_table_is_created(table_name) while result != expected: time.sleep(0.5) result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) @@ -79,15 +88,17 @@ def test_load_and_sync_all_database_tables(started_cluster): NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(50)".format(i)) + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table(cursor, table_name); + instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(50)".format(table_name)) instance.query("CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") assert 'test_database' in instance.query('SHOW DATABASES') for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - cursor.execute('drop table postgresql_replica_{};'.format(i)) + table_name = 'postgresql_replica_{}'.format(i) + check_tables_are_synchronized(table_name); + cursor.execute('drop table {};'.format(table_name)) result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''') assert(int(result) == NUM_TABLES) @@ -113,7 +124,8 @@ def test_replicating_dml(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format(i, i)) for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + table_name = 'postgresql_replica_{}'.format(i) + check_tables_are_synchronized(table_name); for i in range(NUM_TABLES): cursor.execute('UPDATE postgresql_replica_{} SET value = {} * {} WHERE key < 50;'.format(i, i, i)) @@ -225,7 +237,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(50)".format(i)) - if i < NUM_TABLES/2: + if i < int(NUM_TABLES/2): if publication_tables != '': publication_tables += ', ' publication_tables += table_name @@ -239,13 +251,17 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): time.sleep(1) + for i in range(int(NUM_TABLES/2)): + table_name = 'postgresql_replica_{}'.format(i) + assert_nested_table_is_created(table_name) + result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''') - assert(int(result) == NUM_TABLES/2) + assert(int(result) == int(NUM_TABLES/2)) database_tables = instance.query('SHOW TABLES FROM test_database') for i in range(NUM_TABLES): table_name = 'postgresql_replica_{}'.format(i) - if i < NUM_TABLES/2: + if i < int(NUM_TABLES/2): assert table_name in database_tables else: assert table_name not in database_tables @@ -253,7 +269,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): for i in range(NUM_TABLES): table_name = 'postgresql_replica_{}'.format(i) - if i < NUM_TABLES/2: + if i < int(NUM_TABLES/2): check_tables_are_synchronized(table_name); cursor.execute('drop table {};'.format(table_name)) From 44c39d4609186bea6ffb469187270ec769b3e0cc Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 Feb 2021 19:00:28 +0000 Subject: [PATCH 021/105] better --- .../PostgreSQL/PostgreSQLReplicationHandler.cpp | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index bc50e5ab270..2b1b0e03649 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -309,19 +309,7 @@ void PostgreSQLReplicationHandler::shutdownFinal() std::unordered_set PostgreSQLReplicationHandler::fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_) { - auto publication_exist = [&]() - { - auto tx = std::make_shared(*connection_); - bool exist = isPublicationExist(tx); - tx->commit(); - return exist; - }; - - if (publication_exist()) - { - return fetchTablesFromPublication(connection_); - } - else if (tables_list.empty()) + if (tables_list.empty()) { return fetchPostgreSQLTablesList(connection_); } From 46dd137a5a4f7efe9349ae275f810c44ff618b0b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Feb 2021 20:42:18 +0000 Subject: [PATCH 022/105] Fix tests --- src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp | 7 +++++-- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 +- src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp | 1 - .../test_postgresql_replica_database_engine/test.py | 3 ++- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index c2d59ddf516..35590a92709 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -211,8 +211,11 @@ void PostgreSQLReplicaConsumer::readTupleData( value += readInt8(message, pos, size); } - /// TODO: Check for null values and use insertDefaultValue - insertValue(buffer, value, column_idx); + /// For arrays default for null is inserted when converted to clickhouse array + if (value == "NULL") + insertDefaultValue(buffer, column_idx); + else + insertValue(buffer, value, column_idx); LOG_DEBUG(log, "Identifier: {}, column length: {}, value: {}", identifier, col_len, value); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 2b1b0e03649..ba86978cdf0 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -62,7 +62,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() } catch (const pqxx::broken_connection & pqxx_error) { - LOG_ERROR(log, "Unable to set up connection. Reconnection attempt continue. Error message: {}", + LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what()); startup_task->scheduleAfter(reschedule_ms); diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 4348330c832..fe5b0223ef4 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -202,7 +202,6 @@ ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function

name = "tuple"; order_by_expression->arguments = std::make_shared(); - //TODO: check for nullable for (const auto & column : primary_key_columns) order_by_expression->arguments->children.emplace_back(std::make_shared(column.name)); } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 50f7ded4354..8fc2f5c9c69 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -46,10 +46,11 @@ def assert_nested_table_is_created(table_name): def check_tables_are_synchronized(table_name, order_by='key'): + assert_nested_table_is_created(table_name) + expected = instance.query('select * from postgres_database.{} order by {};'.format(table_name, order_by)) result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) - assert_nested_table_is_created(table_name) while result != expected: time.sleep(0.5) result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) From 1480e951798f14a56786d78b839074e35552a6b8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Feb 2021 06:06:37 +0000 Subject: [PATCH 023/105] Fix build, fix arcadia --- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 -- src/Storages/PostgreSQL/PostgreSQLConnection.cpp | 5 +---- src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp | 4 +++- src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h | 10 ++++++++-- src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp | 3 +++ src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h | 2 +- src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp | 5 +++++ src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h | 8 ++++++++ .../PostgreSQL/PostgreSQLReplicationHandler.cpp | 3 +++ src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h | 8 +++++++- src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp | 2 ++ src/Storages/PostgreSQL/StoragePostgreSQLReplica.h | 4 ++++ src/Storages/PostgreSQL/insertPostgreSQLValue.h | 2 +- 13 files changed, 46 insertions(+), 12 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index a3624236f76..8b8b13d031a 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -14,8 +14,6 @@ #include #include -#include - namespace DB { diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp b/src/Storages/PostgreSQL/PostgreSQLConnection.cpp index 668550ec721..d65f98591d5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.cpp @@ -1,9 +1,6 @@ -#if !defined(ARCADIA_BUILD) -#include "config_core.h" -#endif +#include #if USE_LIBPQXX -#include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 35590a92709..64e201106d8 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -1,4 +1,6 @@ #include "PostgreSQLReplicaConsumer.h" + +#if USE_LIBPQXX #include "StoragePostgreSQLReplica.h" #include @@ -526,4 +528,4 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() } - +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index cb1c76829f1..11bba9c8c14 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -1,15 +1,20 @@ #pragma once +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX #include "PostgreSQLConnection.h" #include "PostgreSQLReplicaMetadata.h" -#include "pqxx/pqxx" +#include "insertPostgreSQLValue.h" #include #include #include -#include #include #include +#include "pqxx/pqxx" // Y_IGNORE namespace DB @@ -118,3 +123,4 @@ private: } +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp index 8a45b415ad0..ad9ef4b22d3 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp @@ -1,5 +1,6 @@ #include "PostgreSQLReplicaMetadata.h" +#if USE_LIBPQXX #include #include #include @@ -95,3 +96,5 @@ void PostgreSQLReplicaMetadata::commitMetadata(std::string & lsn, const std::fun } } + +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h index ca7a258e24c..f7e566cce90 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h @@ -10,7 +10,7 @@ class PostgreSQLReplicaMetadata public: PostgreSQLReplicaMetadata(const std::string & metadata_file_path); - void commitMetadata(std::string & lsn, const std::function & syncTableFunc); + void commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc); void readMetadata(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp index aa1fec92ef4..dc714cb5488 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp @@ -1,4 +1,6 @@ #include "PostgreSQLReplicaSettings.h" + +#if USE_LIBPQXX #include #include #include @@ -37,4 +39,7 @@ void PostgreSQLReplicaSettings::loadFromQuery(ASTStorage & storage_def) storage_def.set(storage_def.settings, settings_ast); } } + } + +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h index 1dbd6b0a65b..5ea2a5cd1f6 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h @@ -1,7 +1,13 @@ #pragma once +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX #include + namespace DB { class ASTStorage; @@ -19,3 +25,5 @@ struct PostgreSQLReplicaSettings : public BaseSettings #include #include @@ -342,3 +343,5 @@ PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure( } } + +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index a51c497c21d..5d973ca34fe 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -1,5 +1,10 @@ #pragma once +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX #include "PostgreSQLConnection.h" #include "PostgreSQLReplicaConsumer.h" #include "PostgreSQLReplicaMetadata.h" @@ -75,6 +80,7 @@ private: std::unordered_map nested_storages; }; - } +#endif + diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index fe5b0223ef4..86c1a717905 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -1,5 +1,6 @@ #include "StoragePostgreSQLReplica.h" +#if USE_LIBPQXX #include #include #include @@ -495,3 +496,4 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) } +#endif diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 4059cb744e6..56fc059aa39 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -1,7 +1,10 @@ #pragma once +#if !defined(ARCADIA_BUILD) #include "config_core.h" +#endif +#if USE_LIBPQXX #include "PostgreSQLReplicationHandler.h" #include "PostgreSQLReplicaSettings.h" @@ -100,3 +103,4 @@ private: } +#endif diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.h b/src/Storages/PostgreSQL/insertPostgreSQLValue.h index d9f24247935..dd093cd4c5b 100644 --- a/src/Storages/PostgreSQL/insertPostgreSQLValue.h +++ b/src/Storages/PostgreSQL/insertPostgreSQLValue.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include // Y_IGNORE namespace DB From 9cc63780de2dca946ff012e6d507ed2860241df9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Feb 2021 18:14:05 +0000 Subject: [PATCH 024/105] More correct update query --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 96 ++++++++++++------- .../test.py | 5 +- .../test_storage_postgresql_replica/test.py | 11 ++- 3 files changed, 73 insertions(+), 39 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 64e201106d8..2dd21268c1e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -201,26 +201,34 @@ void PostgreSQLReplicaConsumer::readTupleData( Int16 num_columns = readInt16(message, pos, size); LOG_DEBUG(log, "number of columns {}", num_columns); - for (int column_idx = 0; column_idx < num_columns; ++column_idx) + auto proccess_column_value = [&](Int8 identifier, Int16 column_idx) { - /// 'n' means nullable, 'u' means TOASTed value, 't' means text formatted data - char identifier = readInt8(message, pos, size); - Int32 col_len = readInt32(message, pos, size); - String value; - - for (int i = 0; i < col_len; ++i) + LOG_DEBUG(log, "Identifier: {}", identifier); + switch (identifier) { - value += readInt8(message, pos, size); + case 'n': /// NULL + { + insertDefaultValue(buffer, column_idx); + break; + } + case 't': /// Text formatted value + { + Int32 col_len = readInt32(message, pos, size); + String value; + + for (int i = 0; i < col_len; ++i) + value += readInt8(message, pos, size); + + insertValue(buffer, value, column_idx); + break; + } + case 'u': /// Toasted (unchanged) value TODO:! + break; } + }; - /// For arrays default for null is inserted when converted to clickhouse array - if (value == "NULL") - insertDefaultValue(buffer, column_idx); - else - insertValue(buffer, value, column_idx); - - LOG_DEBUG(log, "Identifier: {}, column length: {}, value: {}", identifier, col_len, value); - } + for (int column_idx = 0; column_idx < num_columns; ++column_idx) + proccess_column_value(readInt8(message, pos, size), column_idx); switch (type) { @@ -240,10 +248,7 @@ void PostgreSQLReplicaConsumer::readTupleData( } case PostgreSQLQuery::UPDATE: { - /// TODO: If table has primary key, skip old value and remove first insert with -1. - // Otherwise use replica identity full (with check) and use first insert. - - if (old_value) + if (old_value) /// Only if replica identity is set to full buffer.columns[num_columns]->insert(Int8(-1)); else buffer.columns[num_columns]->insert(Int8(1)); @@ -302,7 +307,6 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati Int8 replica_identity = readInt8(replication_message, pos, size); Int16 num_columns = readInt16(replication_message, pos, size); - /// TODO: If replica identity is not full, check if there will be a full columns list. LOG_DEBUG(log, "INFO: relation id: {}, namespace: {}, relation name: {}, replica identity: {}, columns number: {}", relation_id, relation_namespace, relation_name, replica_identity, num_columns); @@ -351,28 +355,52 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati throw Exception(ErrorCodes::UNKNOWN_TABLE, "Buffer for table {} does not exist", table_to_insert); } + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT); break; } case 'U': // Update { Int32 relation_id = readInt32(replication_message, pos, size); - Int8 primary_key_or_old_tuple_data = readInt8(replication_message, pos, size); - - LOG_DEBUG(log, "relationID {}, key {} current insert table {}", relation_id, primary_key_or_old_tuple_data, table_to_insert); - - /// TODO: Two cases: based on primary keys and based on replica identity full. - /// Add first one and add a check for second one. + LOG_DEBUG(log, "relationID {}, current insert table {}", relation_id, table_to_insert); auto buffer = buffers.find(table_to_insert); - readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); - - if (pos + 1 < size) + auto proccess_identifier = [&](Int8 identifier) -> bool { - Int8 new_tuple_data = readInt8(replication_message, pos, size); - readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE); - LOG_DEBUG(log, "new tuple data: {}", new_tuple_data); - } + LOG_DEBUG(log, "Identifier: {}", identifier); + bool read_next = true; + switch (identifier) + { + case 'K': /// TODO:! + { + /// Only if changed column(s) are part of replica identity index + break; + } + case 'O': + { + /// Old row. Only of replica identity is set to full. + /// (For the case when a table does not have any primary key.) + /// TODO: Need to find suitable order_by for nested table (Now it throws if no primary key) + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); + break; + } + case 'N': + { + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE); + read_next = false; + break; + } + } + + return read_next; + }; + + /// Read either 'K' or 'O'. Never both of them. Also possible not to get both of them. + bool read_next = proccess_identifier(readInt8(replication_message, pos, size)); + + /// 'N'. Always present, but could come in place of 'K' and 'O'. + if (read_next) + proccess_identifier(readInt8(replication_message, pos, size)); break; } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 8fc2f5c9c69..1faf4924685 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -31,10 +31,11 @@ def create_postgres_db(cursor, name): cursor.execute("CREATE DATABASE {}".format(name)) -def create_postgres_table(cursor, table_name): +def create_postgres_table(cursor, table_name, replica_identity_full=False): cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) cursor.execute(postgres_table_template.format(table_name)) - cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) + if replica_identity_full: + cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) def assert_nested_table_is_created(table_name): diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 57d3b5288fb..bb7ff709b6d 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -13,9 +13,10 @@ instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml' postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( - key Integer NOT NULL, value Integer) + key Integer NOT NULL, value Integer, PRIMARY KEY(key)) """ + def get_postgres_conn(database=False): if database == True: conn_string = "host='localhost' dbname='postgres_database' user='postgres' password='mysecretpassword'" @@ -26,13 +27,17 @@ def get_postgres_conn(database=False): conn.autocommit = True return conn + def create_postgres_db(cursor, name): cursor.execute("CREATE DATABASE {}".format(name)) -def create_postgres_table(cursor, table_name): + +def create_postgres_table(cursor, table_name, replica_identity_full=False): cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) cursor.execute(postgres_table_template.format(table_name)) - cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) + if replica_identity_full: + cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) + def postgresql_replica_check_result(result, check=False, ref_file='test_postgresql_replica.reference'): fpath = p.join(p.dirname(__file__), ref_file) From 3107f82a3b8da5c4e3f7f4b30f890d083eb3b689 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Feb 2021 18:20:52 +0000 Subject: [PATCH 025/105] Fix clang tidy --- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 2 +- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 14 +++++++------- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 2 +- src/Storages/PostgreSQL/StoragePostgreSQLReplica.h | 4 ++-- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 29c3ac7491a..58fedc01e36 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -96,7 +96,7 @@ void DatabasePostgreSQLReplica::startSynchronization() if (storage) { - replication_handler->addStorage(table_name, storage.get()->template as()); + replication_handler->addStorage(table_name, storage->template as()); tables[table_name] = storage; } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 11bba9c8c14..23ab7e22f3e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -83,8 +83,8 @@ private: using Buffers = std::unordered_map; - void insertDefaultValue(BufferData & buffer, size_t column_idx); - void insertValue(BufferData & buffer, const std::string & value, size_t column_idx); + static void insertDefaultValue(BufferData & buffer, size_t column_idx); + static void insertValue(BufferData & buffer, const std::string & value, size_t column_idx); enum class PostgreSQLQuery { @@ -95,11 +95,11 @@ private: void readTupleData(BufferData & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); - void readString(const char * message, size_t & pos, size_t size, String & result); - Int64 readInt64(const char * message, size_t & pos, size_t size); - Int32 readInt32(const char * message, size_t & pos, size_t size); - Int16 readInt16(const char * message, size_t & pos, size_t size); - Int8 readInt8(const char * message, size_t & pos, size_t size); + static void readString(const char * message, size_t & pos, size_t size, String & result); + static Int64 readInt64(const char * message, size_t & pos, size_t size); + static Int32 readInt32(const char * message, size_t & pos, size_t size); + static Int16 readInt16(const char * message, size_t & pos, size_t size); + static Int8 readInt8(const char * message, size_t & pos, size_t size); Poco::Logger * log; std::shared_ptr context; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 86c1a717905..b4b25c3eae9 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -95,7 +95,7 @@ std::string StoragePostgreSQLReplica::getNestedTableName() const std::shared_ptr StoragePostgreSQLReplica::getMaterializedColumnsDeclaration( - const String name, const String type, UInt64 default_value) const + const String name, const String type, UInt64 default_value) { auto column_declaration = std::make_shared(); diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index 56fc059aa39..a0e27ef046d 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -78,8 +78,8 @@ protected: std::unique_ptr replication_settings_); private: - std::shared_ptr getMaterializedColumnsDeclaration( - const String name, const String type, UInt64 default_value) const; + static std::shared_ptr getMaterializedColumnsDeclaration( + const String name, const String type, UInt64 default_value); ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; From ff8b54ffd42750e4646dcccb35e7a43cb7f6f139 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Feb 2021 23:33:01 +0000 Subject: [PATCH 026/105] Make sure postgres table schema changes do not break replication --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 73 +++++++++++++++++-- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 16 ++++ .../test.py | 48 +++++++++++- 3 files changed, 127 insertions(+), 10 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 2dd21268c1e..f0970ff994e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -203,7 +203,8 @@ void PostgreSQLReplicaConsumer::readTupleData( auto proccess_column_value = [&](Int8 identifier, Int16 column_idx) { - LOG_DEBUG(log, "Identifier: {}", identifier); + char id = identifier; + LOG_DEBUG(log, "Identifier: {}", id); switch (identifier) { case 'n': /// NULL @@ -216,7 +217,7 @@ void PostgreSQLReplicaConsumer::readTupleData( Int32 col_len = readInt32(message, pos, size); String value; - for (int i = 0; i < col_len; ++i) + for (Int16 i = 0; i < col_len; ++i) value += readInt8(message, pos, size); insertValue(buffer, value, column_idx); @@ -299,11 +300,10 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati readString(replication_message, pos, size, relation_namespace); readString(replication_message, pos, size, relation_name); - /// TODO: Save relation id (unique to tables) and check if they might be shuffled in current block. - /// If shuffled, store tables based on those id's and insert accordingly. table_to_insert = relation_name; tables_to_sync.insert(table_to_insert); + /// TODO: Add replica identity settings to metadata (needed for update) Int8 replica_identity = readInt8(replication_message, pos, size); Int16 num_columns = readInt16(replication_message, pos, size); @@ -311,10 +311,30 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati "INFO: relation id: {}, namespace: {}, relation name: {}, replica identity: {}, columns number: {}", relation_id, relation_namespace, relation_name, replica_identity, num_columns); - Int8 key; - Int32 data_type_id, type_modifier; + /// Cache table schema data to be able to detect schema changes, because ddl is not + /// replicated with postgresql logical replication protocol, but some table schema info + /// is received if it is the first time we received dml message for given relation in current session or + /// if relation definition has changed since the last relation definition message. + Int8 key; /// Flags. 0 or 1 (if part of the key). Not needed for now. + Int32 data_type_id; + Int32 type_modifier; /// For example, n in varchar(n) + + bool new_relation_definition = false; + if (relation_id_to_name.find(relation_id) == relation_id_to_name.end()) + { + relation_id_to_name.emplace(relation_id, relation_name); + schema_data.emplace(relation_id, SchemaData(num_columns)); + new_relation_definition = true; + } + + auto & current_schema_data = schema_data.find(relation_id)->second; + + if (current_schema_data.number_of_columns != num_columns) + { + markTableAsSkippedUntilReload(relation_id, relation_name); + break; + } - /// TODO: Check here if table structure has changed and, if possible, change table structure and redump table. for (uint16_t i = 0; i < num_columns; ++i) { String column_name; @@ -327,6 +347,20 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati LOG_DEBUG(log, "Key: {}, column name: {}, data type id: {}, type modifier: {}", key, column_name, data_type_id, type_modifier); + + if (new_relation_definition) + { + current_schema_data.column_identifiers.emplace_back(std::make_tuple(data_type_id, type_modifier)); + } + else + { + if (current_schema_data.column_identifiers[i].first != data_type_id + || current_schema_data.column_identifiers[i].second != type_modifier) + { + markTableAsSkippedUntilReload(relation_id, relation_name); + break; + } + } } if (storages.find(table_to_insert) == storages.end()) @@ -345,6 +379,10 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati case 'I': // Insert { Int32 relation_id = readInt32(replication_message, pos, size); + + if (skip_until_reload.find(relation_id) != skip_until_reload.end()) + break; + Int8 new_tuple = readInt8(replication_message, pos, size); LOG_DEBUG(log, "relationID: {}, newTuple: {}, current insert table: {}", relation_id, new_tuple, table_to_insert); @@ -362,12 +400,17 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati case 'U': // Update { Int32 relation_id = readInt32(replication_message, pos, size); + + if (skip_until_reload.find(relation_id) != skip_until_reload.end()) + break; + LOG_DEBUG(log, "relationID {}, current insert table {}", relation_id, table_to_insert); auto buffer = buffers.find(table_to_insert); auto proccess_identifier = [&](Int8 identifier) -> bool { - LOG_DEBUG(log, "Identifier: {}", identifier); + char id = identifier; + LOG_DEBUG(log, "Identifier: {}", id); bool read_next = true; switch (identifier) { @@ -407,6 +450,10 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati case 'D': // Delete { Int32 relation_id = readInt32(replication_message, pos, size); + + if (skip_until_reload.find(relation_id) != skip_until_reload.end()) + break; + Int8 full_replica_identity = readInt8(replication_message, pos, size); LOG_DEBUG(log, "relationID: {}, full replica identity: {}", relation_id, full_replica_identity); @@ -424,6 +471,16 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati } +/// TODO: If some table has a changed structure, we can stop current stream (after remembering last valid WAL position) +/// and advance lsn up to this position. Then make changes to nested table and continue the same way. +void PostgreSQLReplicaConsumer::markTableAsSkippedUntilReload(Int32 relation_id, const String & relation_name) +{ + skip_until_reload.insert(relation_id); + auto & buffer = buffers.find(relation_name)->second; + buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); +} + + void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr tx) { for (const auto & table_name : tables_to_sync) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 23ab7e22f3e..170f85aef7a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -101,6 +101,8 @@ private: static Int16 readInt16(const char * message, size_t & pos, size_t size); static Int8 readInt8(const char * message, size_t & pos, size_t size); + void markTableAsSkippedUntilReload(Int32 relation_id, const String & relation_name); + Poco::Logger * log; std::shared_ptr context; const std::string replication_slot_name, publication_name; @@ -119,6 +121,20 @@ private: Storages storages; Buffers buffers; + + std::unordered_map relation_id_to_name; + + struct SchemaData + { + Int16 number_of_columns; + /// data_type_id and type_modifier + std::vector> column_identifiers; + + SchemaData(Int16 number_of_columns_) : number_of_columns(number_of_columns_) {} + }; + + std::unordered_map schema_data; + std::unordered_set skip_until_reload; }; } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 1faf4924685..f6995355758 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -2,6 +2,7 @@ import pytest import time import psycopg2 import os.path as p +import random from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry @@ -15,6 +16,10 @@ postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( key Integer NOT NULL, value Integer, PRIMARY KEY(key)) """ +postgres_table_template_2 = """ + CREATE TABLE IF NOT EXISTS {} ( + key Integer NOT NULL, value1 Integer, value2 Integer, value3 Integer, PRIMARY KEY(key)) + """ def get_postgres_conn(database=False): if database == True: @@ -31,9 +36,9 @@ def create_postgres_db(cursor, name): cursor.execute("CREATE DATABASE {}".format(name)) -def create_postgres_table(cursor, table_name, replica_identity_full=False): +def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template): cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) - cursor.execute(postgres_table_template.format(table_name)) + cursor.execute(template.format(table_name)) if replica_identity_full: cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) @@ -83,6 +88,7 @@ def postgresql_setup_teardown(): instance.query('DROP TABLE IF EXISTS test.postgresql_replica') +@pytest.mark.timeout(120) def test_load_and_sync_all_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -109,6 +115,7 @@ def test_load_and_sync_all_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') +@pytest.mark.timeout(120) def test_replicating_dml(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -151,6 +158,7 @@ def test_replicating_dml(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') +@pytest.mark.timeout(120) def test_different_data_types(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -190,6 +198,13 @@ def test_different_data_types(started_cluster): check_tables_are_synchronized('test_data_types', 'id'); result = instance.query('SELECT * FROM test_database.test_data_types ORDER BY id LIMIT 1;') assert(result == '0\t-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n') + + for i in range(10): + col = random.choice(['a', 'b', 'c']) + cursor.execute('UPDATE test_data_types SET {} = {};'.format(col, i)) + cursor.execute('''UPDATE test_data_types SET i = '2020-12-12';'''.format(col, i)) + + check_tables_are_synchronized('test_data_types', 'id'); cursor.execute('drop table test_data_types;') instance.query("INSERT INTO postgres_database.test_array_data_type " @@ -227,6 +242,7 @@ def test_different_data_types(started_cluster): assert(result == expected) +@pytest.mark.timeout(120) def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -279,6 +295,34 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') +@pytest.mark.timeout(120) +def test_table_schema_changes(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(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); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(50)".format(i, i, i, i)) + + instance.query( + "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + cursor.execute("ALTER TABLE postgresql_replica_3 DROP COLUMN value2") + + for i in range(NUM_TABLES): + cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) + cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) + + # Wait to check nothing breaks + time.sleep(5) + # TODO + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 0424770f6805fe6a2176fd16675d13620e3d3e21 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 Feb 2021 10:40:59 +0000 Subject: [PATCH 027/105] Handle ddl part 1 --- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 3 +- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 73 +++++------- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 13 +-- .../PostgreSQLReplicationHandler.cpp | 105 ++++++++++++++++-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 15 ++- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 25 ++++- .../PostgreSQL/StoragePostgreSQLReplica.h | 4 +- .../test.py | 15 ++- 8 files changed, 167 insertions(+), 86 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 58fedc01e36..7ce2e47bb02 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -164,7 +164,6 @@ StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, con return table->second; return StoragePtr{}; - } @@ -177,6 +176,7 @@ void DatabasePostgreSQLReplica::createTable(const Context & context, const if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { Base::createTable(context, name, table, query); + /// TODO: Update table cached tables list or not return; } } @@ -188,6 +188,7 @@ void DatabasePostgreSQLReplica::createTable(const Context & context, const template void DatabasePostgreSQLReplica::dropTable(const Context & context, const String & name, bool no_delay) { + /// TODO: If called from non sync thread, add dropped storage to skip list Base::dropTable(context, name, no_delay); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index f0970ff994e..efc5c4614e7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -22,10 +22,6 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -static const auto reschedule_ms = 500; -static const auto max_thread_work_duration_ms = 60000; - - PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( std::shared_ptr context_, PostgreSQLConnectionPtr connection_, @@ -49,13 +45,10 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( { buffers.emplace(table_name, BufferData(storage)); } - - wal_reader_task = context->getSchedulePool().createTask("PostgreSQLReplicaWALReader", [this]{ synchronizationStream(); }); - wal_reader_task->deactivate(); } -void PostgreSQLReplicaConsumer::startSynchronization() +void PostgreSQLReplicaConsumer::readMetadata() { try { @@ -73,35 +66,6 @@ void PostgreSQLReplicaConsumer::startSynchronization() { tryLogCurrentException(__PRETTY_FUNCTION__); } - - wal_reader_task->activateAndSchedule(); -} - - -void PostgreSQLReplicaConsumer::stopSynchronization() -{ - stop_synchronization.store(true); - wal_reader_task->deactivate(); -} - - -void PostgreSQLReplicaConsumer::synchronizationStream() -{ - auto start_time = std::chrono::steady_clock::now(); - - while (!stop_synchronization) - { - if (!readFromReplicationSlot()) - break; - - auto end_time = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - if (duration.count() > max_thread_work_duration_ms) - break; - } - - if (!stop_synchronization) - wal_reader_task->scheduleAfter(reschedule_ms); } @@ -331,8 +295,8 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati if (current_schema_data.number_of_columns != num_columns) { - markTableAsSkippedUntilReload(relation_id, relation_name); - break; + markTableAsSkipped(relation_id, relation_name); + return; } for (uint16_t i = 0; i < num_columns; ++i) @@ -357,8 +321,8 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati if (current_schema_data.column_identifiers[i].first != data_type_id || current_schema_data.column_identifiers[i].second != type_modifier) { - markTableAsSkippedUntilReload(relation_id, relation_name); - break; + markTableAsSkipped(relation_id, relation_name); + return; } } } @@ -380,7 +344,7 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos, size); - if (skip_until_reload.find(relation_id) != skip_until_reload.end()) + if (skip_list.find(relation_id) != skip_list.end()) break; Int8 new_tuple = readInt8(replication_message, pos, size); @@ -401,7 +365,7 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos, size); - if (skip_until_reload.find(relation_id) != skip_until_reload.end()) + if (skip_list.find(relation_id) != skip_list.end()) break; LOG_DEBUG(log, "relationID {}, current insert table {}", relation_id, table_to_insert); @@ -451,7 +415,7 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos, size); - if (skip_until_reload.find(relation_id) != skip_until_reload.end()) + if (skip_list.find(relation_id) != skip_list.end()) break; Int8 full_replica_identity = readInt8(replication_message, pos, size); @@ -471,13 +435,12 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati } -/// TODO: If some table has a changed structure, we can stop current stream (after remembering last valid WAL position) -/// and advance lsn up to this position. Then make changes to nested table and continue the same way. -void PostgreSQLReplicaConsumer::markTableAsSkippedUntilReload(Int32 relation_id, const String & relation_name) +void PostgreSQLReplicaConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name) { - skip_until_reload.insert(relation_id); + skip_list.insert(relation_id); auto & buffer = buffers.find(relation_name)->second; buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); + LOG_DEBUG(log, "Table {} is skipped temporarily", relation_name); } @@ -611,6 +574,20 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() return true; } + +bool PostgreSQLReplicaConsumer::consume(NameSet & skipped_tables) +{ + if (!readFromReplicationSlot() || !skip_list.empty()) + { + for (const auto & relation_id : skip_list) + skipped_tables.insert(relation_id_to_name[relation_id]); + + return false; + } + + return true; +} + } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 170f85aef7a..0f2062214c1 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -10,6 +10,7 @@ #include "insertPostgreSQLValue.h" #include +#include #include #include #include @@ -19,6 +20,7 @@ namespace DB { +using NestedReloadFunc = std::function; class PostgreSQLReplicaConsumer { @@ -35,9 +37,9 @@ public: const size_t max_block_size_, Storages storages_); - void startSynchronization(); + void readMetadata(); - void stopSynchronization(); + bool consume(NameSet & skipped_tables); private: void synchronizationStream(); @@ -101,7 +103,7 @@ private: static Int16 readInt16(const char * message, size_t & pos, size_t size); static Int8 readInt8(const char * message, size_t & pos, size_t size); - void markTableAsSkippedUntilReload(Int32 relation_id, const String & relation_name); + void markTableAsSkipped(Int32 relation_id, const String & relation_name); Poco::Logger * log; std::shared_ptr context; @@ -116,9 +118,6 @@ private: std::string table_to_insert; std::unordered_set tables_to_sync; - BackgroundSchedulePool::TaskHolder wal_reader_task; - std::atomic stop_synchronization = false; - Storages storages; Buffers buffers; @@ -134,7 +133,7 @@ private: }; std::unordered_map schema_data; - std::unordered_set skip_until_reload; + std::unordered_set skip_list; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 440fda0347b..a6b1ca64330 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -15,6 +15,8 @@ namespace DB { static const auto reschedule_ms = 500; +static const auto max_thread_work_duration_ms = 60000; + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, @@ -37,6 +39,9 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); startup_task->deactivate(); + + consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); + consumer_task->deactivate(); } @@ -77,8 +82,8 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() void PostgreSQLReplicationHandler::shutdown() { - if (consumer) - consumer->stopSynchronization(); + stop_synchronization.store(true); + consumer_task->deactivate(); } @@ -95,7 +100,7 @@ void PostgreSQLReplicationHandler::startSynchronization() auto initial_sync = [&]() { createReplicationSlot(ntx, start_lsn, snapshot_name); - loadFromSnapshot(snapshot_name); + loadFromSnapshot(snapshot_name, storages); }; /// Replication slot should be deleted with drop table only and created only once, reused after detach. @@ -124,7 +129,7 @@ void PostgreSQLReplicationHandler::startSynchronization() consumer = std::make_shared( context, - std::move(connection), + connection, replication_slot, publication_name, metadata_path, @@ -132,15 +137,15 @@ void PostgreSQLReplicationHandler::startSynchronization() max_block_size, nested_storages); - consumer->startSynchronization(); + consumer_task->activateAndSchedule(); replication_connection->conn()->close(); } -void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) +void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages) { - for (const auto & storage_data : storages) + for (const auto & storage_data : sync_storages) { try { @@ -159,18 +164,19 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) /// Already connected to needed database, no need to add it to query. query_str = fmt::format("SELECT * FROM {}", storage_data.first); + const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); + auto insert_context = storage_data.second->makeNestedTableContext(); + auto insert = std::make_shared(); insert->table_id = nested_storage->getStorageID(); - auto insert_context = storage_data.second->makeNestedTableContext(); InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); - const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - PostgreSQLBlockInputStream input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); + assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); copyData(input, *block_io.out); storage_data.second->setNestedLoaded(); @@ -188,6 +194,35 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name) } +void PostgreSQLReplicationHandler::consumerFunc() +{ + auto start_time = std::chrono::steady_clock::now(); + NameSet skipped_tables; + + while (!stop_synchronization) + { + bool reschedule = !consumer->consume(skipped_tables); + + if (!skipped_tables.empty()) + { + reloadFromSnapshot(skipped_tables); + skipped_tables.clear(); + } + + if (reschedule) + break; + + auto end_time = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (duration.count() > max_thread_work_duration_ms) + break; + } + + if (!stop_synchronization) + consumer_task->scheduleAfter(reschedule_ms); +} + + bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr tx) { std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); @@ -258,9 +293,16 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, } -void PostgreSQLReplicationHandler::createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name) +void PostgreSQLReplicationHandler::createReplicationSlot( + NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name, bool temporary) { - std::string query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", replication_slot); + std::string query_str; + + if (!temporary) + query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", replication_slot); + else + query_str = fmt::format("CREATE_REPLICATION_SLOT {} TEMPORARY LOGICAL pgoutput EXPORT_SNAPSHOT", replication_slot + "_tmp"); + try { pqxx::result result{ntx->exec(query_str)}; @@ -342,6 +384,45 @@ PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure( return fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true); } + +/// TODO: After temporary replication slot is created, we have a start lsn. In replication stream +/// when get message for a table and this table turn out to be in a skip list, check +/// if current lsn position is >= start lsn position for skipped table. If so, we can +/// remove this table fromm skip list and consume changes without any loss. +std::string PostgreSQLReplicationHandler::reloadFromSnapshot(NameSet & table_names) +{ + String start_lsn; + try + { + auto tx = std::make_shared(*connection->conn()); + Storages sync_storages; + for (const auto & table_name : table_names) + { + auto storage = storages[table_name]; + sync_storages[table_name] = storage; + storage->dropNested(); + } + tx->commit(); + + auto replication_connection = std::make_shared(fmt::format("{} replication=database", connection_str)); + replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); + + auto ntx = std::make_shared(*replication_connection->conn()); + std::string snapshot_name; + createReplicationSlot(ntx, start_lsn, snapshot_name, true); + ntx->commit(); + + loadFromSnapshot(snapshot_name, sync_storages); + + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + return start_lsn; +} + } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5d973ca34fe..bf7b80bbe9b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -42,6 +42,7 @@ public: private: using NontransactionPtr = std::shared_ptr; + using Storages = std::unordered_map; bool isPublicationExist(std::shared_ptr tx); @@ -49,7 +50,7 @@ private: void createPublicationIfNeeded(PostgreSQLConnection::ConnectionPtr connection_); - void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name); + void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name); @@ -59,10 +60,14 @@ private: void startSynchronization(); - void loadFromSnapshot(std::string & snapshot_name); + void consumerFunc(); + + void loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages); std::unordered_set fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_); + std::string reloadFromSnapshot(NameSet & table_names); + Poco::Logger * log; std::shared_ptr context; const std::string database_name, connection_str, metadata_path; @@ -72,11 +77,11 @@ private: PostgreSQLConnectionPtr connection; std::shared_ptr consumer; - BackgroundSchedulePool::TaskHolder startup_task; - std::atomic tables_loaded = false; + BackgroundSchedulePool::TaskHolder startup_task, consumer_task; + std::atomic tables_loaded = false, stop_synchronization = false; bool new_publication_created = false; - std::unordered_map storages; + Storages storages; std::unordered_map nested_storages; }; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index b4b25c3eae9..db3b9afed07 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -172,7 +172,7 @@ ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function

getColumns(); NamesAndTypesList ordinary_columns_and_types; - if (!columns.empty()) + if (!is_postgresql_replica_database) { ordinary_columns_and_types = columns.getOrdinary(); } @@ -243,16 +243,26 @@ ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function

& fetch_table_structure) { - nested_storage = tryGetNested(); + if (nested_loaded) + { + nested_storage = tryGetNested(); - if (nested_storage) - return; + if (nested_storage) + return; + } auto context = makeNestedTableContext(); const auto ast_create = getCreateNestedTableQuery(fetch_table_structure); - InterpreterCreateQuery interpreter(ast_create, context); - interpreter.execute(); + try + { + InterpreterCreateQuery interpreter(ast_create, context); + interpreter.execute(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } nested_storage = getNested(); } @@ -337,6 +347,9 @@ void StoragePostgreSQLReplica::dropNested() auto context = makeNestedTableContext(); auto interpreter = InterpreterDropQuery(ast_drop, context); interpreter.execute(); + + nested_loaded.store(false); + nested_storage = nullptr; } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index a0e27ef046d..d2bb80307fc 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -67,6 +67,8 @@ public: bool isNestedLoaded() { return nested_loaded.load(); } + void dropNested(); + protected: StoragePostgreSQLReplica( const StorageID & table_id_, @@ -87,8 +89,6 @@ private: std::string getNestedTableName() const; - void dropNested(); - std::string remote_table_name; std::shared_ptr global_context; diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index f6995355758..b7ba810f7a2 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -304,23 +304,28 @@ def test_table_schema_changes(started_cluster): for i in range(NUM_TABLES): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i), template=postgres_table_template_2); - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(50)".format(i, i, i, i)) + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) instance.query( "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + 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)) + for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - cursor.execute("ALTER TABLE postgresql_replica_3 DROP COLUMN value2") + expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); + cursor.execute("ALTER TABLE postgresql_replica_4 DROP COLUMN value2") for i in range(NUM_TABLES): cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) - # Wait to check nothing breaks - time.sleep(5) - # TODO + time.sleep(4) + print("Sync check") + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); if __name__ == '__main__': From d0d90538ea8d3d7405791a2d3c5699472f0f76f2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Feb 2021 18:04:10 +0000 Subject: [PATCH 028/105] Tiny fix --- src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index db3b9afed07..f85b9510f50 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -336,6 +336,8 @@ void StoragePostgreSQLReplica::shutdownFinal() void StoragePostgreSQLReplica::dropNested() { + nested_loaded.store(false); + auto table_id = nested_storage->getStorageID(); auto ast_drop = std::make_shared(); @@ -348,7 +350,6 @@ void StoragePostgreSQLReplica::dropNested() auto interpreter = InterpreterDropQuery(ast_drop, context); interpreter.execute(); - nested_loaded.store(false); nested_storage = nullptr; } From 883cc2c0efdca82fc9a324ec003f274c9aeaba83 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Feb 2021 18:37:54 +0000 Subject: [PATCH 029/105] Fixes --- src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 +- src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp | 11 +++++++---- src/Storages/PostgreSQL/StoragePostgreSQLReplica.h | 1 + 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index efc5c4614e7..aecbb7b3bec 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -181,7 +181,7 @@ void PostgreSQLReplicaConsumer::readTupleData( Int32 col_len = readInt32(message, pos, size); String value; - for (Int16 i = 0; i < col_len; ++i) + for (Int32 i = 0; i < col_len; ++i) value += readInt8(message, pos, size); insertValue(buffer, value, column_idx); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index a6b1ca64330..c95b4bacad9 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -398,7 +398,7 @@ std::string PostgreSQLReplicationHandler::reloadFromSnapshot(NameSet & table_nam Storages sync_storages; for (const auto & table_name : table_names) { - auto storage = storages[table_name]; + auto * storage = storages[table_name]; sync_storages[table_name] = storage; storage->dropNested(); } diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index f85b9510f50..c54f9a31cd4 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -336,7 +336,8 @@ void StoragePostgreSQLReplica::shutdownFinal() void StoragePostgreSQLReplica::dropNested() { - nested_loaded.store(false); + std::lock_guard lock(nested_mutex); + nested_loaded = false; auto table_id = nested_storage->getStorageID(); auto ast_drop = std::make_shared(); @@ -373,7 +374,9 @@ Pipe StoragePostgreSQLReplica::read( unsigned num_streams) { /// If initial table sync has not yet finished, nested tables might not be created yet. - if (!nested_loaded) + /// Or nested table might be attempted to get dropped. (Second mutex lock in dropNested()). + std::unique_lock lock(nested_mutex, std::defer_lock); + if (!nested_loaded || !lock.try_lock()) { LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Table {} is not loaded yet", getNestedTableName()); return Pipe(); @@ -383,7 +386,7 @@ Pipe StoragePostgreSQLReplica::read( if (!nested_storage) getNested(); - auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto storage_lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); @@ -428,7 +431,7 @@ Pipe StoragePostgreSQLReplica::read( nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); - pipe.addTableLock(lock); + pipe.addTableLock(storage_lock); if (!expressions->children.empty() && !pipe.empty()) { diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h index d2bb80307fc..4d407f337ad 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h @@ -97,6 +97,7 @@ private: std::atomic nested_loaded = false; StoragePtr nested_storage; + std::mutex nested_mutex; bool is_postgresql_replica_database = false; }; From 427aad80a14734e52f79a4ee3ba4e8607bef092c Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Feb 2021 21:28:14 +0000 Subject: [PATCH 030/105] Avoid table does not exist errors if nested is unavailable --- .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 5 +- .../PostgreSQL/DatabasePostgreSQLReplica.h | 1 - .../PostgreSQL/StoragePostgreSQLReplica.cpp | 118 +++++++++--------- .../test.py | 4 +- 4 files changed, 64 insertions(+), 64 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 7ce2e47bb02..35d808c0dec 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -88,6 +88,7 @@ void DatabasePostgreSQLReplica::startSynchronization() : (global_context.getSettingsRef().max_insert_block_size.value), global_context.getMacros()->expand(settings->postgresql_tables_list.value)); + /// TODO: may be no need to always fetch std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->conn()); for (const auto & table_name : tables_to_replicate) @@ -160,7 +161,9 @@ StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, con } auto table = tables.find(name); - if (table != tables.end() && table->second->as()->isNestedLoaded()) + /// Here it is possible that nested table is temporarily out of reach, but return storage anyway, + /// it will not allow to read if nested is unavailable at the moment + if (table != tables.end()) return table->second; return StoragePtr{}; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h index 24763e697e6..d8cb2ff5a6d 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h @@ -58,7 +58,6 @@ public: void shutdown() override; - private: void startSynchronization(); diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index c54f9a31cd4..0f895c21ae4 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -352,6 +352,7 @@ void StoragePostgreSQLReplica::dropNested() interpreter.execute(); nested_storage = nullptr; + LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Dropped (or temporarily) nested table {}", getNestedTableName()); } @@ -373,79 +374,78 @@ Pipe StoragePostgreSQLReplica::read( size_t max_block_size, unsigned num_streams) { - /// If initial table sync has not yet finished, nested tables might not be created yet. - /// Or nested table might be attempted to get dropped. (Second mutex lock in dropNested()). + /// TODO: are there other places where this lock is needed std::unique_lock lock(nested_mutex, std::defer_lock); - if (!nested_loaded || !lock.try_lock()) + + if (nested_loaded && lock.try_lock()) { - LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Table {} is not loaded yet", getNestedTableName()); - return Pipe(); - } + if (!nested_storage) + getNested(); - /// Should throw if there is no nested storage - if (!nested_storage) - getNested(); + auto storage_lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - auto storage_lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); + Block nested_header = nested_metadata->getSampleBlock(); - const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); + ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); + ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); - Block nested_header = nested_metadata->getSampleBlock(); - ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); - ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - - if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) - { - auto & tables_in_select_query = select_query->tables()->as(); - - if (!tables_in_select_query.children.empty()) + if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) { - auto & tables_element = tables_in_select_query.children[0]->as(); + auto & tables_in_select_query = select_query->tables()->as(); - if (tables_element.table_expression) - tables_element.table_expression->as().final = true; + if (!tables_in_select_query.children.empty()) + { + auto & tables_element = tables_in_select_query.children[0]->as(); + + if (tables_element.table_expression) + tables_element.table_expression->as().final = true; + } } - } - String filter_column_name; - Names require_columns_name = column_names; - ASTPtr expressions = std::make_shared(); - if (column_names_set.empty() || !column_names_set.count(sign_column.name)) - { - require_columns_name.emplace_back(sign_column.name); - - const auto & sign_column_name = std::make_shared(sign_column.name); - const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); - - expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); - filter_column_name = expressions->children.back()->getColumnName(); - - for (const auto & column_name : column_names) - expressions->children.emplace_back(std::make_shared(column_name)); - } - - Pipe pipe = nested_storage->read( - require_columns_name, - nested_metadata, query_info, context, - processed_stage, max_block_size, num_streams); - - pipe.addTableLock(storage_lock); - - if (!expressions->children.empty() && !pipe.empty()) - { - Block pipe_header = pipe.getHeader(); - auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); - ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); - - pipe.addSimpleTransform([&](const Block & header) + String filter_column_name; + Names require_columns_name = column_names; + ASTPtr expressions = std::make_shared(); + if (column_names_set.empty() || !column_names_set.count(sign_column.name)) { - return std::make_shared(header, expression_actions, filter_column_name, false); - }); + require_columns_name.emplace_back(sign_column.name); + + const auto & sign_column_name = std::make_shared(sign_column.name); + const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); + + expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); + filter_column_name = expressions->children.back()->getColumnName(); + + for (const auto & column_name : column_names) + expressions->children.emplace_back(std::make_shared(column_name)); + } + + Pipe pipe = nested_storage->read( + require_columns_name, + nested_metadata, query_info, context, + processed_stage, max_block_size, num_streams); + + pipe.addTableLock(storage_lock); + + if (!expressions->children.empty() && !pipe.empty()) + { + Block pipe_header = pipe.getHeader(); + auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, expression_actions, filter_column_name, false); + }); + } + + return pipe; } - return pipe; + LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Nested table {} is unavailable or is not loaded yet", getNestedTableName()); + return Pipe(); } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index b7ba810f7a2..1b6887ace51 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -316,14 +316,12 @@ def test_table_schema_changes(started_cluster): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); - cursor.execute("ALTER TABLE postgresql_replica_4 DROP COLUMN value2") + cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(random.randint(0, 4))) for i in range(NUM_TABLES): cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) - time.sleep(4) - print("Sync check") for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); From 2e3bdd662e87c2e87970acd1aa2655773117802f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 21 Feb 2021 22:41:18 +0000 Subject: [PATCH 031/105] Handle ddl part 2 --- .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 130 ++++++++++++++---- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 73 ++++++---- .../PostgreSQLReplicationHandler.cpp | 59 +++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 8 +- .../test.py | 13 +- 5 files changed, 197 insertions(+), 86 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index aecbb7b3bec..9c76db7c0e6 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -43,11 +43,36 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( { for (const auto & [table_name, storage] : storages) { - buffers.emplace(table_name, BufferData(storage)); + buffers.emplace(table_name, Buffer(storage)); } } +void PostgreSQLReplicaConsumer::Buffer::fillBuffer(StoragePtr storage) +{ + const auto storage_metadata = storage->getInMemoryMetadataPtr(); + description.init(storage_metadata->getSampleBlock()); + + columns = description.sample_block.cloneEmptyColumns(); + const auto & storage_columns = storage_metadata->getColumns().getAllPhysical(); + auto insert_columns = std::make_shared(); + + assert(description.sample_block.columns() == storage_columns.size()); + size_t idx = 0; + + for (const auto & column : storage_columns) + { + if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) + preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); + idx++; + + insert_columns->children.emplace_back(std::make_shared(column.name)); + } + + columnsAST = std::move(insert_columns); +} + + void PostgreSQLReplicaConsumer::readMetadata() { try @@ -69,7 +94,7 @@ void PostgreSQLReplicaConsumer::readMetadata() } -void PostgreSQLReplicaConsumer::insertValue(BufferData & buffer, const std::string & value, size_t column_idx) +void PostgreSQLReplicaConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); bool is_nullable = buffer.description.types[column_idx].second; @@ -95,7 +120,7 @@ void PostgreSQLReplicaConsumer::insertValue(BufferData & buffer, const std::stri } -void PostgreSQLReplicaConsumer::insertDefaultValue(BufferData & buffer, size_t column_idx) +void PostgreSQLReplicaConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column); @@ -160,7 +185,7 @@ Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos, [ void PostgreSQLReplicaConsumer::readTupleData( - BufferData & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) + Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos, size); LOG_DEBUG(log, "number of columns {}", num_columns); @@ -251,6 +276,7 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp final_lsn = current_lsn; + LOG_DEBUG(log, "Commit lsn: {}", getLSNValue(current_lsn)); break; } case 'O': // Origin @@ -275,18 +301,17 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati "INFO: relation id: {}, namespace: {}, relation name: {}, replica identity: {}, columns number: {}", relation_id, relation_namespace, relation_name, replica_identity, num_columns); - /// Cache table schema data to be able to detect schema changes, because ddl is not - /// replicated with postgresql logical replication protocol, but some table schema info - /// is received if it is the first time we received dml message for given relation in current session or - /// if relation definition has changed since the last relation definition message. + if (!isSyncAllowed(relation_id)) + return; + Int8 key; /// Flags. 0 or 1 (if part of the key). Not needed for now. Int32 data_type_id; Int32 type_modifier; /// For example, n in varchar(n) bool new_relation_definition = false; - if (relation_id_to_name.find(relation_id) == relation_id_to_name.end()) + if (schema_data.find(relation_id) == schema_data.end()) { - relation_id_to_name.emplace(relation_id, relation_name); + relation_id_to_name[relation_id] = relation_name; schema_data.emplace(relation_id, SchemaData(num_columns)); new_relation_definition = true; } @@ -344,8 +369,8 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos, size); - if (skip_list.find(relation_id) != skip_list.end()) - break; + if (!isSyncAllowed(relation_id)) + return; Int8 new_tuple = readInt8(replication_message, pos, size); @@ -365,11 +390,11 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos, size); - if (skip_list.find(relation_id) != skip_list.end()) - break; - LOG_DEBUG(log, "relationID {}, current insert table {}", relation_id, table_to_insert); + if (!isSyncAllowed(relation_id)) + return; + auto buffer = buffers.find(table_to_insert); auto proccess_identifier = [&](Int8 identifier) -> bool { @@ -415,8 +440,8 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati { Int32 relation_id = readInt32(replication_message, pos, size); - if (skip_list.find(relation_id) != skip_list.end()) - break; + if (!isSyncAllowed(relation_id)) + return; Int8 full_replica_identity = readInt8(replication_message, pos, size); @@ -435,15 +460,6 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati } -void PostgreSQLReplicaConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name) -{ - skip_list.insert(relation_id); - auto & buffer = buffers.find(relation_name)->second; - buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); - LOG_DEBUG(log, "Table {} is skipped temporarily", relation_name); -} - - void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr tx) { for (const auto & table_name : tables_to_sync) @@ -505,7 +521,39 @@ String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptrsecond; + if (table_start_lsn.empty()) + return false; + + if (getLSNValue(current_lsn) >= getLSNValue(table_start_lsn)) + { + skip_list.erase(table_with_lsn); + LOG_DEBUG(log, "Sync is allowed for relation id: {}", relation_id); + + return true; + } + + return false; +} + + +void PostgreSQLReplicaConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name) +{ + skip_list.insert({relation_id, ""}); + schema_data.erase(relation_id); + auto & buffer = buffers.find(relation_name)->second; + buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); + LOG_DEBUG(log, "Table {} is skipped temporarily", relation_name); +} + + +/// Read binary changes from replication slot via COPY command (starting from current lsn in a slot). bool PostgreSQLReplicaConsumer::readFromReplicationSlot() { std::shared_ptr tx; @@ -545,6 +593,7 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() slot_empty = false; current_lsn = (*row)[0]; + LOG_DEBUG(log, "Current lsn: {}", getLSNValue(current_lsn)); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } @@ -575,12 +624,15 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() } -bool PostgreSQLReplicaConsumer::consume(NameSet & skipped_tables) +bool PostgreSQLReplicaConsumer::consume(std::vector> & skipped_tables) { if (!readFromReplicationSlot() || !skip_list.empty()) { - for (const auto & relation_id : skip_list) - skipped_tables.insert(relation_id_to_name[relation_id]); + for (const auto & [relation_id, lsn] : skip_list) + { + if (lsn.empty()) + skipped_tables.emplace_back(std::make_pair(relation_id, relation_id_to_name[relation_id])); + } return false; } @@ -588,6 +640,24 @@ bool PostgreSQLReplicaConsumer::consume(NameSet & skipped_tables) return true; } + +void PostgreSQLReplicaConsumer::updateNested(const String & table_name, StoragePtr nested_storage) +{ + storages[table_name] = nested_storage; + auto & buffer = buffers.find(table_name)->second; + buffer.fillBuffer(nested_storage); +} + + +void PostgreSQLReplicaConsumer::updateSkipList(const std::unordered_map & tables_with_lsn) +{ + for (const auto & [relation_id, lsn] : tables_with_lsn) + { + if (!lsn.empty()) + skip_list[relation_id] = lsn; /// start_lsn + } +} + } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 0f2062214c1..f8b214db4b7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -18,9 +18,13 @@ #include "pqxx/pqxx" // Y_IGNORE +/// TODO: There is ALTER PUBLICATION command to dynamically add and remove tables for replicating (the command is transactional). +/// This can also be supported. (Probably, if in a replication stream comes a relation name, which does not currenly +/// exist in CH, it can be loaded from snapshot and handled the same way as some ddl by comparing lsn positions of wal, +/// but there is the case that a known table has been just renamed, then the previous version might be just dropped by user). + namespace DB { -using NestedReloadFunc = std::function; class PostgreSQLReplicaConsumer { @@ -39,7 +43,11 @@ public: void readMetadata(); - bool consume(NameSet & skipped_tables); + bool consume(std::vector> & skipped_tables); + + void updateNested(const String & table_name, StoragePtr nested_table); + + void updateSkipList(const std::unordered_map & tables_with_lsn); private: void synchronizationStream(); @@ -52,7 +60,9 @@ private: void processReplicationMessage(const char * replication_message, size_t size); - struct BufferData + bool isSyncAllowed(Int32 relation_id); + + struct Buffer { ExternalResultDescription description; MutableColumns columns; @@ -60,33 +70,14 @@ private: /// Needed for insertPostgreSQLValue() method to parse array std::unordered_map array_info; - BufferData(StoragePtr storage) - { - const auto storage_metadata = storage->getInMemoryMetadataPtr(); - description.init(storage_metadata->getSampleBlock()); - columns = description.sample_block.cloneEmptyColumns(); - const auto & storage_columns = storage_metadata->getColumns().getAllPhysical(); - auto insert_columns = std::make_shared(); - size_t idx = 0; - assert(description.sample_block.columns() == storage_columns.size()); - - for (const auto & column : storage_columns) - { - if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) - preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); - idx++; - - insert_columns->children.emplace_back(std::make_shared(column.name)); - } - - columnsAST = std::move(insert_columns); - } + Buffer(StoragePtr storage) { fillBuffer(storage); } + void fillBuffer(StoragePtr storage); }; - using Buffers = std::unordered_map; + using Buffers = std::unordered_map; - static void insertDefaultValue(BufferData & buffer, size_t column_idx); - static void insertValue(BufferData & buffer, const std::string & value, size_t column_idx); + static void insertDefaultValue(Buffer & buffer, size_t column_idx); + static void insertValue(Buffer & buffer, const std::string & value, size_t column_idx); enum class PostgreSQLQuery { @@ -95,7 +86,7 @@ private: DELETE }; - void readTupleData(BufferData & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); + void readTupleData(Buffer & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); static void readString(const char * message, size_t & pos, size_t size, String & result); static Int64 readInt64(const char * message, size_t & pos, size_t size); @@ -105,6 +96,14 @@ private: void markTableAsSkipped(Int32 relation_id, const String & relation_name); + /// lsn - log sequnce nuumber, like wal offset (64 bit). + Int64 getLSNValue(const std::string & lsn) + { + Int64 upper_half, lower_half; + std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); + return (upper_half << 32) + lower_half; + } + Poco::Logger * log; std::shared_ptr context; const std::string replication_slot_name, publication_name; @@ -116,6 +115,8 @@ private: const size_t max_block_size; std::string table_to_insert; + + /// List of tables which need to be synced after last replication stream. std::unordered_set tables_to_sync; Storages storages; @@ -132,8 +133,22 @@ private: SchemaData(Int16 number_of_columns_) : number_of_columns(number_of_columns_) {} }; + /// Cache for table schema data to be able to detect schema changes, because ddl is not + /// replicated with postgresql logical replication protocol, but some table schema info + /// is received if it is the first time we received dml message for given relation in current session or + /// if relation definition has changed since the last relation definition message. std::unordered_map schema_data; - std::unordered_set skip_list; + + /// skip_list contains relation ids for tables on which ddl was perfomed, which can break synchronization. + /// This breaking changes are detected in replication stream in according replication message and table is added to skip list. + /// After it is finished, a temporary replication slot is created with 'export snapshot' option, and start_lsn is returned. + /// Skipped tables are reloaded from snapshot (nested tables are also updated). Afterwards, if a replication message is + /// related to a table in a skip_list, we compare current lsn with start_lsn, which was returned with according snapshot. + /// If current_lsn >= table_start_lsn, we can safely remove table from skip list and continue its synchronization. + std::unordered_map skip_list; + + /// Mapping from table name which is currently in a skip_list to a table_start_lsn for future comparison with current_lsn. + //NameToNameMap start_lsn_for_skipped; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index c95b4bacad9..83afe658661 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -120,8 +120,15 @@ void PostgreSQLReplicationHandler::startSynchronization() { for (const auto & [table_name, storage] : storages) { - nested_storages[table_name] = storage->getNested(); - storage->setNestedLoaded(); + try + { + nested_storages[table_name] = storage->getNested(); + storage->setNestedLoaded(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } } @@ -143,13 +150,15 @@ void PostgreSQLReplicationHandler::startSynchronization() } -void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages) +NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages) { + NameSet success_tables; for (const auto & storage_data : sync_storages) { try { auto tx = std::make_shared(*connection->conn()); + const auto & table_name = storage_data.first; /// Specific isolation level is required to read from snapshot. tx->set_variable("transaction_isolation", "'repeatable read'"); @@ -157,7 +166,7 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); - storage_data.second->createNestedIfNeeded([&]() { return fetchTableStructure(tx, storage_data.first); }); + storage_data.second->createNestedIfNeeded([&]() { return fetchTableStructure(tx, table_name); }); auto nested_storage = storage_data.second->getNested(); /// Load from snapshot, which will show table state before creation of replication slot. @@ -180,7 +189,12 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, copyData(input, *block_io.out); storage_data.second->setNestedLoaded(); - nested_storages[storage_data.first] = nested_storage; + nested_storages[table_name] = nested_storage; + + /// This is needed if this method is called from reloadFromSnapshot() method below. + success_tables.insert(table_name); + if (consumer) + consumer->updateNested(table_name, nested_storage); } catch (Exception & e) { @@ -191,23 +205,21 @@ void PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, } LOG_DEBUG(log, "Table dump end"); + return success_tables; } void PostgreSQLReplicationHandler::consumerFunc() { auto start_time = std::chrono::steady_clock::now(); - NameSet skipped_tables; + std::vector> skipped_tables; while (!stop_synchronization) { bool reschedule = !consumer->consume(skipped_tables); if (!skipped_tables.empty()) - { - reloadFromSnapshot(skipped_tables); - skipped_tables.clear(); - } + consumer->updateSkipList(reloadFromSnapshot(skipped_tables)); if (reschedule) break; @@ -350,7 +362,7 @@ void PostgreSQLReplicationHandler::shutdownFinal() } -std::unordered_set PostgreSQLReplicationHandler::fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_) +NameSet PostgreSQLReplicationHandler::fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_) { if (tables_list.empty()) { @@ -364,7 +376,7 @@ std::unordered_set PostgreSQLReplicationHandler::fetchRequiredTable } -std::unordered_set PostgreSQLReplicationHandler::fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_) +NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_) { std::string query = fmt::format("SELECT tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); std::unordered_set tables; @@ -385,19 +397,17 @@ PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure( } -/// TODO: After temporary replication slot is created, we have a start lsn. In replication stream -/// when get message for a table and this table turn out to be in a skip list, check -/// if current lsn position is >= start lsn position for skipped table. If so, we can -/// remove this table fromm skip list and consume changes without any loss. -std::string PostgreSQLReplicationHandler::reloadFromSnapshot(NameSet & table_names) +std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapshot( + const std::vector> & relation_data) { - String start_lsn; + std::unordered_map tables_start_lsn; try { auto tx = std::make_shared(*connection->conn()); Storages sync_storages; - for (const auto & table_name : table_names) + for (const auto & relation : relation_data) { + const auto & table_name = relation.second; auto * storage = storages[table_name]; sync_storages[table_name] = storage; storage->dropNested(); @@ -408,19 +418,24 @@ std::string PostgreSQLReplicationHandler::reloadFromSnapshot(NameSet & table_nam replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); auto ntx = std::make_shared(*replication_connection->conn()); - std::string snapshot_name; + std::string snapshot_name, start_lsn; createReplicationSlot(ntx, start_lsn, snapshot_name, true); ntx->commit(); - loadFromSnapshot(snapshot_name, sync_storages); + auto success_tables = loadFromSnapshot(snapshot_name, sync_storages); + for (const auto & relation : relation_data) + { + if (success_tables.find(relation.second) != success_tables.end()) + tables_start_lsn[relation.first] = start_lsn; + } } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } - return start_lsn; + return tables_start_lsn; } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index bf7b80bbe9b..f428ed9720a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -36,7 +36,7 @@ public: void addStorage(const std::string & table_name, StoragePostgreSQLReplica * storage); - std::unordered_set fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_); + NameSet fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_); PostgreSQLTableStructure fetchTableStructure(std::shared_ptr tx, const std::string & table_name); @@ -62,11 +62,11 @@ private: void consumerFunc(); - void loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages); + NameSet loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages); - std::unordered_set fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_); + NameSet fetchTablesFromPublication(PostgreSQLConnection::ConnectionPtr connection_); - std::string reloadFromSnapshot(NameSet & table_names); + std::unordered_map reloadFromSnapshot(const std::vector> & relation_data); Poco::Logger * log; std::shared_ptr context; diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 1b6887ace51..ec82a1050c2 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -316,7 +316,9 @@ def test_table_schema_changes(started_cluster): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); - cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(random.randint(0, 4))) + + altered_table = random.randint(0, 4) + cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) for i in range(NUM_TABLES): cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) @@ -325,6 +327,15 @@ def test_table_schema_changes(started_cluster): for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + for i in range(NUM_TABLES): + if i != altered_table: + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) + else: + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + if __name__ == '__main__': cluster.start() From 8a48bb24ce683c843f9fe85b2c1f98f698bac0c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 21 Feb 2021 23:13:58 +0000 Subject: [PATCH 032/105] Fix typos, comments --- src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp | 3 --- src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h | 4 ++-- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 4 ++-- src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp | 1 - 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 35d808c0dec..9105dc9ba25 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -88,7 +88,6 @@ void DatabasePostgreSQLReplica::startSynchronization() : (global_context.getSettingsRef().max_insert_block_size.value), global_context.getMacros()->expand(settings->postgresql_tables_list.value)); - /// TODO: may be no need to always fetch std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->conn()); for (const auto & table_name : tables_to_replicate) @@ -179,7 +178,6 @@ void DatabasePostgreSQLReplica::createTable(const Context & context, const if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { Base::createTable(context, name, table, query); - /// TODO: Update table cached tables list or not return; } } @@ -191,7 +189,6 @@ void DatabasePostgreSQLReplica::createTable(const Context & context, const template void DatabasePostgreSQLReplica::dropTable(const Context & context, const String & name, bool no_delay) { - /// TODO: If called from non sync thread, add dropped storage to skip list Base::dropTable(context, name, no_delay); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index f8b214db4b7..55f8a949cd1 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -19,7 +19,7 @@ /// TODO: There is ALTER PUBLICATION command to dynamically add and remove tables for replicating (the command is transactional). -/// This can also be supported. (Probably, if in a replication stream comes a relation name, which does not currenly +/// This can also be supported. (Probably, if in a replication stream comes a relation name, which does not currently /// exist in CH, it can be loaded from snapshot and handled the same way as some ddl by comparing lsn positions of wal, /// but there is the case that a known table has been just renamed, then the previous version might be just dropped by user). @@ -139,7 +139,7 @@ private: /// if relation definition has changed since the last relation definition message. std::unordered_map schema_data; - /// skip_list contains relation ids for tables on which ddl was perfomed, which can break synchronization. + /// skip_list contains relation ids for tables on which ddl was performed, which can break synchronization. /// This breaking changes are detected in replication stream in according replication message and table is added to skip list. /// After it is finished, a temporary replication slot is created with 'export snapshot' option, and start_lsn is returned. /// Skipped tables are reloaded from snapshot (nested tables are also updated). Afterwards, if a replication message is diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 83afe658661..eccaa3c7acf 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -420,9 +420,9 @@ std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapsh auto ntx = std::make_shared(*replication_connection->conn()); std::string snapshot_name, start_lsn; createReplicationSlot(ntx, start_lsn, snapshot_name, true); - ntx->commit(); - + /// This snapshot is valid up to the end of the transaction, which exported it. auto success_tables = loadFromSnapshot(snapshot_name, sync_storages); + ntx->commit(); for (const auto & relation : relation_data) { diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 0f895c21ae4..4b2e746a557 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -374,7 +374,6 @@ Pipe StoragePostgreSQLReplica::read( size_t max_block_size, unsigned num_streams) { - /// TODO: are there other places where this lock is needed std::unique_lock lock(nested_mutex, std::defer_lock); if (nested_loaded && lock.try_lock()) From ace76bb7fa32133c26264ac693581b98d41f6305 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Feb 2021 12:35:53 +0000 Subject: [PATCH 033/105] Better, remove debug logs --- src/Core/Settings.h | 1 + .../PostgreSQL/DatabasePostgreSQLReplica.cpp | 7 +- src/Interpreters/InterpreterCreateQuery.cpp | 6 + .../PostgreSQL/PostgreSQLReplicaConsumer.cpp | 287 ++++++++++-------- .../PostgreSQL/PostgreSQLReplicaConsumer.h | 18 +- .../PostgreSQL/PostgreSQLReplicaSettings.h | 5 +- .../PostgreSQLReplicationHandler.cpp | 35 ++- .../PostgreSQL/PostgreSQLReplicationHandler.h | 5 +- .../PostgreSQL/StoragePostgreSQLReplica.cpp | 9 +- .../configs/users.xml | 8 + .../test.py | 41 ++- .../test_storage_postgresql_replica/test.py | 25 +- 12 files changed, 264 insertions(+), 183 deletions(-) create mode 100644 tests/integration/test_postgresql_replica_database_engine/configs/users.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2d7b7811390..aaf5da3bab7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -402,6 +402,7 @@ class IColumn; M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ + M(Bool, allow_experimental_database_postgresql_replica, false, "Allow to create database with Engine=PostgreSQLReplica(...).", 0) \ M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp index 9105dc9ba25..2b491e62fab 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp @@ -83,10 +83,11 @@ void DatabasePostgreSQLReplica::startSynchronization() connection->conn_str(), metadata_path + METADATA_SUFFIX, std::make_shared(global_context), - settings->postgresql_max_block_size.changed - ? settings->postgresql_max_block_size.value + settings->postgresql_replica_max_block_size.changed + ? settings->postgresql_replica_max_block_size.value : (global_context.getSettingsRef().max_insert_block_size.value), - global_context.getMacros()->expand(settings->postgresql_tables_list.value)); + settings->postgresql_replica_allow_minimal_ddl, true, + settings->postgresql_replica_tables_list.value); std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->conn()); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 2b1dddde78c..46af167eaa4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -214,6 +214,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } + if (create.storage->engine->name == "PostgreSQLReplica" && !context.getSettingsRef().allow_experimental_database_postgresql_replica && !internal) + { + throw Exception("PostgreSQLReplica is an experimental database engine. " + "Enable allow_experimental_database_postgresql_replica to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); + } + DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context); if (create.uuid != UUIDHelpers::Nil) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp index 9c76db7c0e6..91e48e9c358 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp @@ -30,6 +30,8 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, + bool allow_minimal_ddl_, + bool is_postgresql_replica_database_engine_, Storages storages_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) , context(context_) @@ -39,6 +41,8 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( , connection(std::move(connection_)) , current_lsn(start_lsn) , max_block_size(max_block_size_) + , allow_minimal_ddl(allow_minimal_ddl_) + , is_postgresql_replica_database_engine(is_postgresql_replica_database_engine_) , storages(storages_) { for (const auto & [table_name, storage] : storages) @@ -188,12 +192,9 @@ void PostgreSQLReplicaConsumer::readTupleData( Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos, size); - LOG_DEBUG(log, "number of columns {}", num_columns); auto proccess_column_value = [&](Int8 identifier, Int16 column_idx) { - char id = identifier; - LOG_DEBUG(log, "Identifier: {}", id); switch (identifier) { case 'n': /// NULL @@ -212,7 +213,10 @@ void PostgreSQLReplicaConsumer::readTupleData( insertValue(buffer, value, column_idx); break; } - case 'u': /// Toasted (unchanged) value TODO:! + case 'u': /// TOAST value && unchanged at the same time. Actual value is not sent. + /// TOAST values are not supported. (TOAST values are values that are considered in postgres + /// to be too large to be stored directly) + insertDefaultValue(buffer, column_idx); break; } }; @@ -238,7 +242,8 @@ void PostgreSQLReplicaConsumer::readTupleData( } case PostgreSQLQuery::UPDATE: { - if (old_value) /// Only if replica identity is set to full + /// Process old value in case changed value is a primary key. + if (old_value) buffer.columns[num_columns]->insert(Int8(-1)); else buffer.columns[num_columns]->insert(Int8(1)); @@ -258,8 +263,6 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati size_t pos = 2; char type = readInt8(replication_message, pos, size); - LOG_DEBUG(log, "Type of replication message: {}", type); - switch (type) { case 'B': // Begin @@ -268,103 +271,6 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp break; } - case 'C': // Commit - { - readInt8(replication_message, pos, size); /// unused flags - readInt64(replication_message, pos, size); /// Int64 commit lsn - readInt64(replication_message, pos, size); /// Int64 transaction end lsn - readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp - - final_lsn = current_lsn; - LOG_DEBUG(log, "Commit lsn: {}", getLSNValue(current_lsn)); - break; - } - case 'O': // Origin - break; - case 'R': // Relation - { - Int32 relation_id = readInt32(replication_message, pos, size); - - String relation_namespace, relation_name; - - readString(replication_message, pos, size, relation_namespace); - readString(replication_message, pos, size, relation_name); - - table_to_insert = relation_name; - tables_to_sync.insert(table_to_insert); - - /// TODO: Add replica identity settings to metadata (needed for update) - Int8 replica_identity = readInt8(replication_message, pos, size); - Int16 num_columns = readInt16(replication_message, pos, size); - - LOG_DEBUG(log, - "INFO: relation id: {}, namespace: {}, relation name: {}, replica identity: {}, columns number: {}", - relation_id, relation_namespace, relation_name, replica_identity, num_columns); - - if (!isSyncAllowed(relation_id)) - return; - - Int8 key; /// Flags. 0 or 1 (if part of the key). Not needed for now. - Int32 data_type_id; - Int32 type_modifier; /// For example, n in varchar(n) - - bool new_relation_definition = false; - if (schema_data.find(relation_id) == schema_data.end()) - { - relation_id_to_name[relation_id] = relation_name; - schema_data.emplace(relation_id, SchemaData(num_columns)); - new_relation_definition = true; - } - - auto & current_schema_data = schema_data.find(relation_id)->second; - - if (current_schema_data.number_of_columns != num_columns) - { - markTableAsSkipped(relation_id, relation_name); - return; - } - - for (uint16_t i = 0; i < num_columns; ++i) - { - String column_name; - key = readInt8(replication_message, pos, size); - readString(replication_message, pos, size, column_name); - - data_type_id = readInt32(replication_message, pos, size); - type_modifier = readInt32(replication_message, pos, size); - - LOG_DEBUG(log, - "Key: {}, column name: {}, data type id: {}, type modifier: {}", - key, column_name, data_type_id, type_modifier); - - if (new_relation_definition) - { - current_schema_data.column_identifiers.emplace_back(std::make_tuple(data_type_id, type_modifier)); - } - else - { - if (current_schema_data.column_identifiers[i].first != data_type_id - || current_schema_data.column_identifiers[i].second != type_modifier) - { - markTableAsSkipped(relation_id, relation_name); - return; - } - } - } - - if (storages.find(table_to_insert) == storages.end()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Storage for table {} does not exist, but is included in replication stream", table_to_insert); - } - - [[maybe_unused]] auto buffer_iter = buffers.find(table_to_insert); - assert(buffer_iter != buffers.end()); - - break; - } - case 'Y': // Type - break; case 'I': // Insert { Int32 relation_id = readInt32(replication_message, pos, size); @@ -373,46 +279,44 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati return; Int8 new_tuple = readInt8(replication_message, pos, size); + const auto & table_name = relation_id_to_name[relation_id]; + auto buffer = buffers.find(table_name); - LOG_DEBUG(log, "relationID: {}, newTuple: {}, current insert table: {}", relation_id, new_tuple, table_to_insert); - - auto buffer = buffers.find(table_to_insert); if (buffer == buffers.end()) - { - throw Exception(ErrorCodes::UNKNOWN_TABLE, - "Buffer for table {} does not exist", table_to_insert); - } + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Buffer for table {} does not exist", table_name); + + if (new_tuple) + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT); - readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT); break; } case 'U': // Update { Int32 relation_id = readInt32(replication_message, pos, size); - LOG_DEBUG(log, "relationID {}, current insert table {}", relation_id, table_to_insert); - if (!isSyncAllowed(relation_id)) return; - auto buffer = buffers.find(table_to_insert); + const auto & table_name = relation_id_to_name[relation_id]; + auto buffer = buffers.find(table_name); + auto proccess_identifier = [&](Int8 identifier) -> bool { - char id = identifier; - LOG_DEBUG(log, "Identifier: {}", id); bool read_next = true; switch (identifier) { - case 'K': /// TODO:! + case 'K': { - /// Only if changed column(s) are part of replica identity index + /// Only if changed column(s) are part of replica identity index (for now it can be only + /// be primary key - default values for replica identity index). In this case, first comes a tuple + /// with old replica identity indexes and all other values will come as nulls. Then comes a full new row. + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); break; } case 'O': { - /// Old row. Only of replica identity is set to full. - /// (For the case when a table does not have any primary key.) - /// TODO: Need to find suitable order_by for nested table (Now it throws if no primary key) + /// Old row. Only if replica identity is set to full. (For the case when a table does not have any + /// primary key, for now not supported, requires to find suitable order by key(s) for nested table.) readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); break; } @@ -443,14 +347,118 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati if (!isSyncAllowed(relation_id)) return; - Int8 full_replica_identity = readInt8(replication_message, pos, size); + /// 0 or 1 if replica identity is set to full. For now only default replica identity is supported (with primary keys). + readInt8(replication_message, pos, size); - LOG_DEBUG(log, "relationID: {}, full replica identity: {}", relation_id, full_replica_identity); - - auto buffer = buffers.find(table_to_insert); + const auto & table_name = relation_id_to_name[relation_id]; + auto buffer = buffers.find(table_name); readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE); + break; } + case 'C': // Commit + { + readInt8(replication_message, pos, size); /// unused flags + readInt64(replication_message, pos, size); /// Int64 commit lsn + readInt64(replication_message, pos, size); /// Int64 transaction end lsn + readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp + + final_lsn = current_lsn; + LOG_DEBUG(log, "Commit lsn: {}", getLSNValue(current_lsn)); /// Will be removed + + break; + } + case 'R': // Relation + { + Int32 relation_id = readInt32(replication_message, pos, size); + + String relation_namespace, relation_name; + + readString(replication_message, pos, size, relation_namespace); + readString(replication_message, pos, size, relation_name); + + if (!isSyncAllowed(relation_id)) + return; + + /// 'd' - default (primary key if any) + /// 'n' - nothing + /// 'f' - all columns (set replica identity full) + /// 'i' - user defined index with indisreplident set + /// For database engine now supported only 'd', for table engine 'f' is also allowed. + char replica_identity = readInt8(replication_message, pos, size); + + if (replica_identity != 'd' && (replica_identity != 'f' || is_postgresql_replica_database_engine)) + { + LOG_WARNING(log, + "Table has replica identity {} - not supported. " + "For database engine only default (with primary keys) replica identity is supported." + "For table engine full replica identity is also supported. Table will be skipped."); + markTableAsSkipped(relation_id, relation_name); + return; + } + + Int16 num_columns = readInt16(replication_message, pos, size); + + Int32 data_type_id; + Int32 type_modifier; /// For example, n in varchar(n) + + bool new_relation_definition = false; + if (schema_data.find(relation_id) == schema_data.end()) + { + relation_id_to_name[relation_id] = relation_name; + schema_data.emplace(relation_id, SchemaData(num_columns)); + new_relation_definition = true; + } + + auto & current_schema_data = schema_data.find(relation_id)->second; + + if (current_schema_data.number_of_columns != num_columns) + { + markTableAsSkipped(relation_id, relation_name); + return; + } + + for (uint16_t i = 0; i < num_columns; ++i) + { + String column_name; + readInt8(replication_message, pos, size); /// Marks column as part of replica identity index + readString(replication_message, pos, size, column_name); + + data_type_id = readInt32(replication_message, pos, size); + type_modifier = readInt32(replication_message, pos, size); + + if (new_relation_definition) + { + current_schema_data.column_identifiers.emplace_back(std::make_tuple(data_type_id, type_modifier)); + } + else + { + if (current_schema_data.column_identifiers[i].first != data_type_id + || current_schema_data.column_identifiers[i].second != type_modifier) + { + markTableAsSkipped(relation_id, relation_name); + return; + } + } + } + + if (storages.find(relation_name) == storages.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Storage for table {} does not exist, but is included in replication stream", relation_name); + } + + [[maybe_unused]] auto buffer_iter = buffers.find(relation_name); + assert(buffer_iter != buffers.end()); + + tables_to_sync.insert(relation_name); + + break; + } + case 'O': // Origin + break; + case 'Y': // Type + break; case 'T': // Truncate break; default: @@ -533,8 +541,10 @@ bool PostgreSQLReplicaConsumer::isSyncAllowed(Int32 relation_id) if (getLSNValue(current_lsn) >= getLSNValue(table_start_lsn)) { + LOG_TRACE(log, "Synchronization is resumed for table: {} (start_lsn: {})", + relation_id_to_name[relation_id], table_start_lsn); + skip_list.erase(table_with_lsn); - LOG_DEBUG(log, "Sync is allowed for relation id: {}", relation_id); return true; } @@ -549,7 +559,10 @@ void PostgreSQLReplicaConsumer::markTableAsSkipped(Int32 relation_id, const Stri schema_data.erase(relation_id); auto & buffer = buffers.find(relation_name)->second; buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); - LOG_DEBUG(log, "Table {} is skipped temporarily", relation_name); + if (!allow_minimal_ddl) + LOG_WARNING(log, "Table {} is skipped, because table schema has changed", relation_name); + else + LOG_TRACE(log, "Table {} is skipped temporarily. ID: {}", relation_name, relation_id); } @@ -593,7 +606,6 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() slot_empty = false; current_lsn = (*row)[0]; - LOG_DEBUG(log, "Current lsn: {}", getLSNValue(current_lsn)); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } @@ -619,19 +631,26 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() return false; } - syncTables(tx); + if (!tables_to_sync.empty()) + { + syncTables(tx); + } + return true; } bool PostgreSQLReplicaConsumer::consume(std::vector> & skipped_tables) { - if (!readFromReplicationSlot() || !skip_list.empty()) + if (!readFromReplicationSlot()) { - for (const auto & [relation_id, lsn] : skip_list) + if (allow_minimal_ddl && !skip_list.empty()) { - if (lsn.empty()) - skipped_tables.emplace_back(std::make_pair(relation_id, relation_id_to_name[relation_id])); + for (const auto & [relation_id, lsn] : skip_list) + { + if (lsn.empty()) + skipped_tables.emplace_back(std::make_pair(relation_id, relation_id_to_name[relation_id])); + } } return false; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 55f8a949cd1..720e2cf72d5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -39,19 +39,19 @@ public: const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, + bool allow_minimal_ddl_, + bool is_postgresql_replica_database_engine_, Storages storages_); void readMetadata(); bool consume(std::vector> & skipped_tables); - void updateNested(const String & table_name, StoragePtr nested_table); + void updateNested(const String & table_name, StoragePtr nested_storage); void updateSkipList(const std::unordered_map & tables_with_lsn); private: - void synchronizationStream(); - bool readFromReplicationSlot(); void syncTables(std::shared_ptr tx); @@ -99,9 +99,9 @@ private: /// lsn - log sequnce nuumber, like wal offset (64 bit). Int64 getLSNValue(const std::string & lsn) { - Int64 upper_half, lower_half; - std::sscanf(lsn.data(), "%lX/%lX", &upper_half, &lower_half); - return (upper_half << 32) + lower_half; + UInt32 upper_half, lower_half; + std::sscanf(lsn.data(), "%X/%X", &upper_half, &lower_half); + return (static_cast(upper_half) << 32) + lower_half; } Poco::Logger * log; @@ -113,6 +113,7 @@ private: std::string current_lsn, final_lsn; const size_t max_block_size; + bool allow_minimal_ddl, is_postgresql_replica_database_engine; std::string table_to_insert; @@ -145,10 +146,9 @@ private: /// Skipped tables are reloaded from snapshot (nested tables are also updated). Afterwards, if a replication message is /// related to a table in a skip_list, we compare current lsn with start_lsn, which was returned with according snapshot. /// If current_lsn >= table_start_lsn, we can safely remove table from skip list and continue its synchronization. + /// No needed message, related to reloaded table will be missed, because messages are not consumed in the meantime, + /// i.e. we will not miss the first start_lsn position for reloaded table. std::unordered_map skip_list; - - /// Mapping from table name which is currently in a skip_list to a table_start_lsn for future comparison with current_lsn. - //NameToNameMap start_lsn_for_skipped; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h index 5ea2a5cd1f6..0f084ac6108 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h @@ -14,8 +14,9 @@ namespace DB #define LIST_OF_POSTGRESQL_REPLICA_SETTINGS(M) \ - M(UInt64, postgresql_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ - M(String, postgresql_tables_list, "", "List of tables for PostgreSQLReplica database engine", 0) \ + M(UInt64, postgresql_replica_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ + M(String, postgresql_replica_tables_list, "", "List of tables for PostgreSQLReplica database engine", 0) \ + M(Bool, postgresql_replica_allow_minimal_ddl, 0, "Allow to track minimal possible ddl. By default, table after ddl will get into a skip list", 0) \ DECLARE_SETTINGS_TRAITS(PostgreSQLReplicaSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index eccaa3c7acf..ee15c1ec13d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -24,6 +24,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & metadata_path_, std::shared_ptr context_, const size_t max_block_size_, + bool allow_minimal_ddl_, + bool is_postgresql_replica_database_engine_, const String tables_list_) : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) , context(context_) @@ -31,6 +33,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , connection_str(conn_str) , metadata_path(metadata_path_) , max_block_size(max_block_size_) + , allow_minimal_ddl(allow_minimal_ddl_) + , is_postgresql_replica_database_engine(is_postgresql_replica_database_engine_) , tables_list(tables_list_) , connection(std::make_shared(conn_str)) { @@ -113,7 +117,7 @@ void PostgreSQLReplicationHandler::startSynchronization() /// In case of some failure, the following cases are possible (since publication and replication slot are reused): /// 1. If replication slot exists and metadata file (where last synced version is written) does not exist, it is not ok. /// 2. If created a new publication and replication slot existed before it was created, it is not ok. - dropReplicationSlot(ntx, replication_slot); + dropReplicationSlot(ntx); initial_sync(); } else @@ -142,6 +146,8 @@ void PostgreSQLReplicationHandler::startSynchronization() metadata_path, start_lsn, max_block_size, + allow_minimal_ddl, + is_postgresql_replica_database_engine, nested_storages); consumer_task->activateAndSchedule(); @@ -310,32 +316,41 @@ void PostgreSQLReplicationHandler::createReplicationSlot( { std::string query_str; - if (!temporary) - query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", replication_slot); + std::string slot_name; + if (temporary) + slot_name = replication_slot + "_tmp"; else - query_str = fmt::format("CREATE_REPLICATION_SLOT {} TEMPORARY LOGICAL pgoutput EXPORT_SNAPSHOT", replication_slot + "_tmp"); + slot_name = replication_slot; + + query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", slot_name); try { pqxx::result result{ntx->exec(query_str)}; start_lsn = result[0][1].as(); snapshot_name = result[0][2].as(); - LOG_TRACE(log, "Created replication slot: {}, start lsn: {}, snapshot: {}", - replication_slot, start_lsn, snapshot_name); + LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, start_lsn); } catch (Exception & e) { - e.addMessage("while creating PostgreSQL replication slot {}", replication_slot); + e.addMessage("while creating PostgreSQL replication slot {}", slot_name); throw; } } -void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, std::string & slot_name) +void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, bool temporary) { + std::string slot_name; + if (temporary) + slot_name = replication_slot + "_tmp"; + else + slot_name = replication_slot; + std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); + ntx->exec(query_str); - LOG_TRACE(log, "Dropped replication slot {}", slot_name); + LOG_TRACE(log, "Dropped replication slot: {}", slot_name); } @@ -356,7 +371,7 @@ void PostgreSQLReplicationHandler::shutdownFinal() dropPublication(tx); if (isReplicationSlotExist(tx, replication_slot)) - dropReplicationSlot(tx, replication_slot); + dropReplicationSlot(tx); tx->commit(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index f428ed9720a..506335d8eb2 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -26,6 +26,8 @@ public: const std::string & metadata_path_, std::shared_ptr context_, const size_t max_block_size_, + bool allow_minimal_ddl_, + bool is_postgresql_replica_database_engine_, const String tables_list = ""); void startup(); @@ -52,7 +54,7 @@ private: void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); - void dropReplicationSlot(NontransactionPtr tx, std::string & slot_name); + void dropReplicationSlot(NontransactionPtr tx, bool temporary = false); void dropPublication(NontransactionPtr ntx); @@ -72,6 +74,7 @@ private: std::shared_ptr context; const std::string database_name, connection_str, metadata_path; const size_t max_block_size; + bool allow_minimal_ddl, is_postgresql_replica_database_engine; std::string tables_list, replication_slot, publication_name; PostgreSQLConnectionPtr connection; diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp index 4b2e746a557..dbacc995c67 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp @@ -61,11 +61,10 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( connection_str, metadata_path, global_context, - replication_settings->postgresql_max_block_size.changed - ? replication_settings->postgresql_max_block_size.value - : (global_context->getSettingsRef().max_insert_block_size.value) - - ); + replication_settings->postgresql_replica_max_block_size.changed + ? replication_settings->postgresql_replica_max_block_size.value + : global_context->getSettingsRef().max_insert_block_size.value, + replication_settings->postgresql_replica_allow_minimal_ddl.value, false); } diff --git a/tests/integration/test_postgresql_replica_database_engine/configs/users.xml b/tests/integration/test_postgresql_replica_database_engine/configs/users.xml new file mode 100644 index 00000000000..948093dbf4c --- /dev/null +++ b/tests/integration/test_postgresql_replica_database_engine/configs/users.xml @@ -0,0 +1,8 @@ + + + + + 1 + + + diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index ec82a1050c2..9d5686175ab 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -10,7 +10,7 @@ from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True) +instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], user_configs = ['configs/users.xml'], with_postgres=True) postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( @@ -88,7 +88,7 @@ def postgresql_setup_teardown(): instance.query('DROP TABLE IF EXISTS test.postgresql_replica') -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_load_and_sync_all_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -115,7 +115,7 @@ def test_load_and_sync_all_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_replicating_dml(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -158,7 +158,7 @@ def test_replicating_dml(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_different_data_types(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -242,7 +242,7 @@ def test_different_data_types(started_cluster): assert(result == expected) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -263,7 +263,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query(''' CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS postgresql_tables_list = '{}'; + SETTINGS postgresql_replica_tables_list = '{}'; '''.format(publication_tables)) assert 'test_database' in instance.query('SHOW DATABASES') @@ -295,7 +295,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -307,7 +307,10 @@ def test_table_schema_changes(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) instance.query( - "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + """CREATE DATABASE test_database + ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + SETTINGS postgresql_replica_allow_minimal_ddl = 1; + """) 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)) @@ -336,6 +339,28 @@ def test_table_schema_changes(started_cluster): for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + for i in range(NUM_TABLES): + cursor.execute('drop table postgresql_replica_{};'.format(i)) + + instance.query("DROP DATABASE test_database") + + +@pytest.mark.timeout(120) +def test_changing_replica_identity_value(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)") + + instance.query( + "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)") + check_tables_are_synchronized('postgresql_replica'); + cursor.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") + check_tables_are_synchronized('postgresql_replica'); + if __name__ == '__main__': cluster.start() diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index bb7ff709b6d..2ca07092607 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -70,7 +70,7 @@ def rabbitmq_setup_teardown(): instance.query('DROP TABLE IF EXISTS test.postgresql_replica') -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_initial_load_from_snapshot(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -93,7 +93,7 @@ def test_initial_load_from_snapshot(started_cluster): postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_no_connection_at_startup(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -120,7 +120,7 @@ def test_no_connection_at_startup(started_cluster): postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_detach_attach_is_ok(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -145,13 +145,16 @@ def test_detach_attach_is_ok(started_cluster): instance.query('DETACH TABLE test.postgresql_replica') instance.query('ATTACH TABLE test.postgresql_replica') - time.sleep(0.5) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + while postgresql_replica_check_result(result) == False: + time.sleep(0.5) + result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') + cursor.execute('DROP TABLE postgresql_replica;') postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_replicating_insert_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -192,7 +195,7 @@ def test_replicating_insert_queries(started_cluster): postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_replicating_delete_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -230,7 +233,7 @@ def test_replicating_delete_queries(started_cluster): postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_replicating_update_queries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -261,7 +264,7 @@ def test_replicating_update_queries(started_cluster): postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_resume_from_written_version(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -303,7 +306,7 @@ def test_resume_from_written_version(started_cluster): postgresql_replica_check_result(result, True) -@pytest.mark.timeout(120) +@pytest.mark.timeout(320) def test_many_replication_messages(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -318,7 +321,7 @@ def test_many_replication_messages(started_cluster): PRIMARY KEY(key)) ENGINE = PostgreSQLReplica( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - SETTINGS postgresql_max_block_size = 50000; + SETTINGS postgresql_replica_max_block_size = 50000; ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') @@ -361,7 +364,7 @@ def test_many_replication_messages(started_cluster): cursor.execute('DROP TABLE postgresql_replica;') -@pytest.mark.timeout(180) +@pytest.mark.timeout(320) def test_connection_loss(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() From 28c0a64c07b0bab34d8464a632e8740a0388213e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Feb 2021 13:05:29 +0000 Subject: [PATCH 034/105] Comment --- src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h | 5 ----- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h | 5 +++++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h index 720e2cf72d5..37b36d4bfc5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h @@ -18,11 +18,6 @@ #include "pqxx/pqxx" // Y_IGNORE -/// TODO: There is ALTER PUBLICATION command to dynamically add and remove tables for replicating (the command is transactional). -/// This can also be supported. (Probably, if in a replication stream comes a relation name, which does not currently -/// exist in CH, it can be loaded from snapshot and handled the same way as some ddl by comparing lsn positions of wal, -/// but there is the case that a known table has been just renamed, then the previous version might be just dropped by user). - namespace DB { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 506335d8eb2..7b9605be5dc 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -14,6 +14,11 @@ namespace DB { +/// IDEA: There is ALTER PUBLICATION command to dynamically add and remove tables for replicating (the command is transactional). +/// (Probably, if in a replication stream comes a relation name, which does not currently +/// exist in CH, it can be loaded via snapshot while stream is stopped and then comparing wal positions with +/// current lsn and table start lsn. + class StoragePostgreSQLReplica; class PostgreSQLReplicationHandler From 87c740730bcbcd1a093f4b3899ebc7bca713c81b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Mar 2021 09:58:10 +0000 Subject: [PATCH 035/105] Rename to MaterializePostgreSQL --- src/Databases/DatabaseFactory.cpp | 16 ++--- ....cpp => DatabaseMaterializePostgreSQL.cpp} | 42 ++++++------- ...lica.h => DatabaseMaterializePostgreSQL.h} | 12 ++-- src/Interpreters/InterpreterDropQuery.cpp | 4 +- ....cpp => MaterializePostgreSQLConsumer.cpp} | 44 +++++++------- ...umer.h => MaterializePostgreSQLConsumer.h} | 8 +-- ....cpp => MaterializePostgreSQLMetadata.cpp} | 14 ++--- ...data.h => MaterializePostgreSQLMetadata.h} | 4 +- ....cpp => MaterializePostgreSQLSettings.cpp} | 6 +- ...ings.h => MaterializePostgreSQLSettings.h} | 6 +- .../PostgreSQLReplicationHandler.cpp | 8 +-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 13 ++-- ...a.cpp => StorageMaterializePostgreSQL.cpp} | 60 +++++++++---------- ...plica.h => StorageMaterializePostgreSQL.h} | 16 ++--- src/Storages/registerStorages.cpp | 4 +- 15 files changed, 128 insertions(+), 129 deletions(-) rename src/Databases/PostgreSQL/{DatabasePostgreSQLReplica.cpp => DatabaseMaterializePostgreSQL.cpp} (77%) rename src/Databases/PostgreSQL/{DatabasePostgreSQLReplica.h => DatabaseMaterializePostgreSQL.h} (84%) rename src/Storages/PostgreSQL/{PostgreSQLReplicaConsumer.cpp => MaterializePostgreSQLConsumer.cpp} (91%) rename src/Storages/PostgreSQL/{PostgreSQLReplicaConsumer.h => MaterializePostgreSQLConsumer.h} (97%) rename src/Storages/PostgreSQL/{PostgreSQLReplicaMetadata.cpp => MaterializePostgreSQLMetadata.cpp} (79%) rename src/Storages/PostgreSQL/{PostgreSQLReplicaMetadata.h => MaterializePostgreSQLMetadata.h} (81%) rename src/Storages/PostgreSQL/{PostgreSQLReplicaSettings.cpp => MaterializePostgreSQLSettings.cpp} (77%) rename src/Storages/PostgreSQL/{PostgreSQLReplicaSettings.h => MaterializePostgreSQLSettings.h} (71%) rename src/Storages/PostgreSQL/{StoragePostgreSQLReplica.cpp => StorageMaterializePostgreSQL.cpp} (87%) rename src/Storages/PostgreSQL/{StoragePostgreSQLReplica.h => StorageMaterializePostgreSQL.h} (83%) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index f153b792caf..d5147b5539b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -36,9 +36,9 @@ #if USE_LIBPQXX #include // Y_IGNORE -#include +#include #include -#include +#include #endif namespace DB @@ -101,14 +101,14 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const UUID & uuid = create.uuid; bool engine_may_have_arguments = engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "Lazy" || - engine_name == "Replicated" || engine_name == "PostgreSQL" || engine_name == "PostgreSQLReplica"; + engine_name == "Replicated" || engine_name == "PostgreSQL" || engine_name == "MaterializePostgreSQL"; if (engine_define->engine->arguments && !engine_may_have_arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || engine_define->sample_by; - bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "PostgreSQLReplica"; + bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializePostgreSQL"; if (has_unexpected_element || (!may_have_settings && engine_define->settings)) throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); @@ -254,7 +254,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared( context, metadata_path, engine_define, database_name, postgres_database_name, connection, use_table_cache); } - else if (engine_name == "PostgreSQLReplica") + else if (engine_name == "MaterializePostgreSQL") { const ASTFunction * engine = engine_define->engine; @@ -279,21 +279,21 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String auto connection = std::make_shared( postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password); - auto postgresql_replica_settings = std::make_unique(); + auto postgresql_replica_settings = std::make_unique(); if (engine_define->settings) postgresql_replica_settings->loadFromQuery(*engine_define); if (create.uuid == UUIDHelpers::Nil) { - return std::make_shared>( + return std::make_shared>( context, metadata_path, uuid, engine_define, database_name, postgres_database_name, connection, std::move(postgresql_replica_settings)); } else { - return std::make_shared>( + return std::make_shared>( context, metadata_path, uuid, engine_define, database_name, postgres_database_name, connection, std::move(postgresql_replica_settings)); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp similarity index 77% rename from src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp rename to src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 2b491e62fab..62c00dfd4a2 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -1,9 +1,9 @@ -#include +#include #if USE_LIBPQXX #include -#include +#include #include #include @@ -30,7 +30,7 @@ namespace DB static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; template<> -DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( +DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const Context & context, const String & metadata_path_, UUID /* uuid */, @@ -38,11 +38,11 @@ DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( const String & database_name_, const String & postgres_database_name, PostgreSQLConnectionPtr connection_, - std::unique_ptr settings_) + std::unique_ptr settings_) : DatabaseOrdinary( database_name_, metadata_path_, "data/" + escapeForFileName(database_name_) + "/", - "DatabasePostgreSQLReplica (" + database_name_ + ")", context) - , log(&Poco::Logger::get("PostgreSQLReplicaDatabaseEngine")) + "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context) + , log(&Poco::Logger::get("MaterializePostgreSQLDatabaseEngine")) , global_context(context.getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) @@ -55,7 +55,7 @@ DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( template<> -DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( +DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const Context & context, const String & metadata_path_, UUID uuid, @@ -63,8 +63,8 @@ DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( const String & database_name_, const String & postgres_database_name, PostgreSQLConnectionPtr connection_, - std::unique_ptr settings_) - : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabasePostgreSQLReplica (" + database_name_ + ")", context) + std::unique_ptr settings_) + : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context) , global_context(context.getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) @@ -76,7 +76,7 @@ DatabasePostgreSQLReplica::DatabasePostgreSQLReplica( template -void DatabasePostgreSQLReplica::startSynchronization() +void DatabaseMaterializePostgreSQL::startSynchronization() { replication_handler = std::make_unique( remote_database_name, @@ -97,7 +97,7 @@ void DatabasePostgreSQLReplica::startSynchronization() if (storage) { - replication_handler->addStorage(table_name, storage->template as()); + replication_handler->addStorage(table_name, storage->template as()); tables[table_name] = storage; } } @@ -108,19 +108,19 @@ void DatabasePostgreSQLReplica::startSynchronization() template -StoragePtr DatabasePostgreSQLReplica::getStorage(const String & name) +StoragePtr DatabaseMaterializePostgreSQL::getStorage(const String & name) { auto storage = tryGetTable(name, global_context); if (storage) return storage; - return StoragePostgreSQLReplica::create(StorageID(database_name, name), StoragePtr{}, global_context); + return StorageMaterializePostgreSQL::create(StorageID(database_name, name), StoragePtr{}, global_context); } template -void DatabasePostgreSQLReplica::shutdown() +void DatabaseMaterializePostgreSQL::shutdown() { if (replication_handler) replication_handler->shutdown(); @@ -128,7 +128,7 @@ void DatabasePostgreSQLReplica::shutdown() template -void DatabasePostgreSQLReplica::loadStoredObjects( +void DatabaseMaterializePostgreSQL::loadStoredObjects( Context & context, bool has_force_restore_data_flag, bool force_attach) { Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach); @@ -149,7 +149,7 @@ void DatabasePostgreSQLReplica::loadStoredObjects( template -StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, const Context & context) const +StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const Context & context) const { if (context.hasQueryContext()) { @@ -171,7 +171,7 @@ StoragePtr DatabasePostgreSQLReplica::tryGetTable(const String & name, con template -void DatabasePostgreSQLReplica::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializePostgreSQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) { if (context.hasQueryContext()) { @@ -188,14 +188,14 @@ void DatabasePostgreSQLReplica::createTable(const Context & context, const template -void DatabasePostgreSQLReplica::dropTable(const Context & context, const String & name, bool no_delay) +void DatabaseMaterializePostgreSQL::dropTable(const Context & context, const String & name, bool no_delay) { Base::dropTable(context, name, no_delay); } template -void DatabasePostgreSQLReplica::drop(const Context & context) +void DatabaseMaterializePostgreSQL::drop(const Context & context) { if (replication_handler) { @@ -214,13 +214,13 @@ void DatabasePostgreSQLReplica::drop(const Context & context) template -DatabaseTablesIteratorPtr DatabasePostgreSQLReplica::getTablesIterator( +DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( const Context & /* context */, const DatabaseOnDisk::FilterByNameFunction & /* filter_by_table_name */) { Tables nested_tables; for (const auto & [table_name, storage] : tables) { - auto nested_storage = storage->template as()->tryGetNested(); + auto nested_storage = storage->template as()->tryGetNested(); if (nested_storage) nested_tables[table_name] = nested_storage; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h similarity index 84% rename from src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h rename to src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index d8cb2ff5a6d..fd6d5982fdf 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQLReplica.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -7,7 +7,7 @@ #if USE_LIBPQXX #include -#include +#include #include #include @@ -25,11 +25,11 @@ using PostgreSQLConnectionPtr = std::shared_ptr; template -class DatabasePostgreSQLReplica : public Base +class DatabaseMaterializePostgreSQL : public Base { public: - DatabasePostgreSQLReplica( + DatabaseMaterializePostgreSQL( const Context & context, const String & metadata_path_, UUID uuid, @@ -37,9 +37,9 @@ public: const String & dbname_, const String & postgres_dbname, PostgreSQLConnectionPtr connection_, - std::unique_ptr settings_); + std::unique_ptr settings_); - String getEngineName() const override { return "PostgreSQLReplica"; } + String getEngineName() const override { return "MaterializePostgreSQL"; } String getMetadataPath() const override { return metadata_path; } @@ -69,7 +69,7 @@ private: ASTPtr database_engine_define; String database_name, remote_database_name; PostgreSQLConnectionPtr connection; - std::unique_ptr settings; + std::unique_ptr settings; std::shared_ptr replication_handler; std::map tables; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 575f5b43a32..1a282b6ef5d 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -23,7 +23,7 @@ #endif #if USE_LIBPQXX -# include +# include #endif namespace DB @@ -192,7 +192,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat #if USE_LIBPQXX if (table->getName() == "PostgreSQLReplica") - table->as()->shutdownFinal(); + table->as()->shutdownFinal(); #endif TableExclusiveLockHolder table_lock; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp similarity index 91% rename from src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp rename to src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 91e48e9c358..8e7dc81647e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -1,7 +1,7 @@ -#include "PostgreSQLReplicaConsumer.h" +#include "MaterializePostgreSQLConsumer.h" #if USE_LIBPQXX -#include "StoragePostgreSQLReplica.h" +#include "StorageMaterializePostgreSQL.h" #include #include @@ -22,7 +22,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( +MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( std::shared_ptr context_, PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, @@ -52,7 +52,7 @@ PostgreSQLReplicaConsumer::PostgreSQLReplicaConsumer( } -void PostgreSQLReplicaConsumer::Buffer::fillBuffer(StoragePtr storage) +void MaterializePostgreSQLConsumer::Buffer::fillBuffer(StoragePtr storage) { const auto storage_metadata = storage->getInMemoryMetadataPtr(); description.init(storage_metadata->getSampleBlock()); @@ -77,7 +77,7 @@ void PostgreSQLReplicaConsumer::Buffer::fillBuffer(StoragePtr storage) } -void PostgreSQLReplicaConsumer::readMetadata() +void MaterializePostgreSQLConsumer::readMetadata() { try { @@ -98,7 +98,7 @@ void PostgreSQLReplicaConsumer::readMetadata() } -void PostgreSQLReplicaConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx) +void MaterializePostgreSQLConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); bool is_nullable = buffer.description.types[column_idx].second; @@ -124,14 +124,14 @@ void PostgreSQLReplicaConsumer::insertValue(Buffer & buffer, const std::string & } -void PostgreSQLReplicaConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx) +void MaterializePostgreSQLConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column); } -void PostgreSQLReplicaConsumer::readString(const char * message, size_t & pos, size_t size, String & result) +void MaterializePostgreSQLConsumer::readString(const char * message, size_t & pos, size_t size, String & result) { assert(size > pos + 2); char current = unhex2(message + pos); @@ -145,7 +145,7 @@ void PostgreSQLReplicaConsumer::readString(const char * message, size_t & pos, s } -Int32 PostgreSQLReplicaConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 8); Int32 result = (UInt32(unhex2(message + pos)) << 24) @@ -157,7 +157,7 @@ Int32 PostgreSQLReplicaConsumer::readInt32(const char * message, size_t & pos, [ } -Int16 PostgreSQLReplicaConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int16 MaterializePostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 4); Int16 result = (UInt32(unhex2(message + pos)) << 8) @@ -167,7 +167,7 @@ Int16 PostgreSQLReplicaConsumer::readInt16(const char * message, size_t & pos, [ } -Int8 PostgreSQLReplicaConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int8 MaterializePostgreSQLConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 2); Int8 result = unhex2(message + pos); @@ -176,7 +176,7 @@ Int8 PostgreSQLReplicaConsumer::readInt8(const char * message, size_t & pos, [[m } -Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 16); Int64 result = (UInt64(unhex4(message + pos)) << 48) @@ -188,7 +188,7 @@ Int64 PostgreSQLReplicaConsumer::readInt64(const char * message, size_t & pos, [ } -void PostgreSQLReplicaConsumer::readTupleData( +void MaterializePostgreSQLConsumer::readTupleData( Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos, size); @@ -257,7 +257,7 @@ void PostgreSQLReplicaConsumer::readTupleData( /// https://www.postgresql.org/docs/13/protocol-logicalrep-message-formats.html -void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replication_message, size_t size) +void MaterializePostgreSQLConsumer::processReplicationMessage(const char * replication_message, size_t size) { /// Skip '\x' size_t pos = 2; @@ -468,7 +468,7 @@ void PostgreSQLReplicaConsumer::processReplicationMessage(const char * replicati } -void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr tx) +void MaterializePostgreSQLConsumer::syncTables(std::shared_ptr tx) { for (const auto & table_name : tables_to_sync) { @@ -517,7 +517,7 @@ void PostgreSQLReplicaConsumer::syncTables(std::shared_ptr } -String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr tx) +String MaterializePostgreSQLConsumer::advanceLSN(std::shared_ptr tx) { std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn); pqxx::result result{tx->exec(query_str)}; @@ -529,7 +529,7 @@ String PostgreSQLReplicaConsumer::advanceLSN(std::shared_ptr tx; bool slot_empty = true; @@ -640,7 +640,7 @@ bool PostgreSQLReplicaConsumer::readFromReplicationSlot() } -bool PostgreSQLReplicaConsumer::consume(std::vector> & skipped_tables) +bool MaterializePostgreSQLConsumer::consume(std::vector> & skipped_tables) { if (!readFromReplicationSlot()) { @@ -660,7 +660,7 @@ bool PostgreSQLReplicaConsumer::consume(std::vector> & } -void PostgreSQLReplicaConsumer::updateNested(const String & table_name, StoragePtr nested_storage) +void MaterializePostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage) { storages[table_name] = nested_storage; auto & buffer = buffers.find(table_name)->second; @@ -668,7 +668,7 @@ void PostgreSQLReplicaConsumer::updateNested(const String & table_name, StorageP } -void PostgreSQLReplicaConsumer::updateSkipList(const std::unordered_map & tables_with_lsn) +void MaterializePostgreSQLConsumer::updateSkipList(const std::unordered_map & tables_with_lsn) { for (const auto & [relation_id, lsn] : tables_with_lsn) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h similarity index 97% rename from src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h rename to src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 37b36d4bfc5..74dd5d67ccf 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -6,7 +6,7 @@ #if USE_LIBPQXX #include "PostgreSQLConnection.h" -#include "PostgreSQLReplicaMetadata.h" +#include "MaterializePostgreSQLMetadata.h" #include "insertPostgreSQLValue.h" #include @@ -21,12 +21,12 @@ namespace DB { -class PostgreSQLReplicaConsumer +class MaterializePostgreSQLConsumer { public: using Storages = std::unordered_map; - PostgreSQLReplicaConsumer( + MaterializePostgreSQLConsumer( std::shared_ptr context_, PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, @@ -103,7 +103,7 @@ private: std::shared_ptr context; const std::string replication_slot_name, publication_name; - PostgreSQLReplicaMetadata metadata; + MaterializePostgreSQLMetadata metadata; PostgreSQLConnectionPtr connection; std::string current_lsn, final_lsn; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp similarity index 79% rename from src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp rename to src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp index ad9ef4b22d3..5cc68b429c0 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp @@ -1,4 +1,4 @@ -#include "PostgreSQLReplicaMetadata.h" +#include "MaterializePostgreSQLMetadata.h" #if USE_LIBPQXX #include @@ -12,7 +12,7 @@ namespace DB { -PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadata_file_path) +MaterializePostgreSQLMetadata::MaterializePostgreSQLMetadata(const std::string & metadata_file_path) : metadata_file(metadata_file_path) , tmp_metadata_file(metadata_file_path + ".tmp") , last_version(1) @@ -20,7 +20,7 @@ PostgreSQLReplicaMetadata::PostgreSQLReplicaMetadata(const std::string & metadat } -void PostgreSQLReplicaMetadata::readMetadata() +void MaterializePostgreSQLMetadata::readMetadata() { if (Poco::File(metadata_file).exists()) { @@ -41,13 +41,13 @@ void PostgreSQLReplicaMetadata::readMetadata() last_lsn = actual_lsn; } - LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaMetadata"), + LOG_DEBUG(&Poco::Logger::get("MaterializePostgreSQLMetadata"), "Last written version is {}. (From metadata file {})", last_version, metadata_file); } } -void PostgreSQLReplicaMetadata::writeMetadata(bool append_metadata) +void MaterializePostgreSQLMetadata::writeMetadata(bool append_metadata) { WriteBufferFromFile out(tmp_metadata_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); @@ -69,7 +69,7 @@ void PostgreSQLReplicaMetadata::writeMetadata(bool append_metadata) /// While data is received, version is updated. Before table sync, write last version to tmp file. /// Then sync data to table and rename tmp to non-tmp. -void PostgreSQLReplicaMetadata::commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc) +void MaterializePostgreSQLMetadata::commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc) { std::string actual_lsn; last_lsn = lsn; @@ -90,7 +90,7 @@ void PostgreSQLReplicaMetadata::commitMetadata(std::string & lsn, const std::fun if (actual_lsn != last_lsn) { writeMetadata(true); - LOG_WARNING(&Poco::Logger::get("PostgreSQLReplicaMetadata"), + LOG_WARNING(&Poco::Logger::get("MaterializePostgreSQLMetadata"), "Last written LSN {} is not equal to actual LSN {}", last_lsn, actual_lsn); } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h similarity index 81% rename from src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h rename to src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h index f7e566cce90..d09adb61363 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaMetadata.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h @@ -5,10 +5,10 @@ namespace DB { -class PostgreSQLReplicaMetadata +class MaterializePostgreSQLMetadata { public: - PostgreSQLReplicaMetadata(const std::string & metadata_file_path); + MaterializePostgreSQLMetadata(const std::string & metadata_file_path); void commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp similarity index 77% rename from src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp rename to src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp index dc714cb5488..48fe61b4182 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp @@ -1,4 +1,4 @@ -#include "PostgreSQLReplicaSettings.h" +#include "MaterializePostgreSQLSettings.h" #if USE_LIBPQXX #include @@ -15,9 +15,9 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(PostgreSQLReplicaSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) -void PostgreSQLReplicaSettings::loadFromQuery(ASTStorage & storage_def) +void MaterializePostgreSQLSettings::loadFromQuery(ASTStorage & storage_def) { if (storage_def.settings) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h similarity index 71% rename from src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h rename to src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h index 0f084ac6108..0df618f513e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaSettings.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h @@ -15,12 +15,12 @@ namespace DB #define LIST_OF_POSTGRESQL_REPLICA_SETTINGS(M) \ M(UInt64, postgresql_replica_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ - M(String, postgresql_replica_tables_list, "", "List of tables for PostgreSQLReplica database engine", 0) \ + M(String, postgresql_replica_tables_list, "", "List of tables for MaterializePostgreSQL database engine", 0) \ M(Bool, postgresql_replica_allow_minimal_ddl, 0, "Allow to track minimal possible ddl. By default, table after ddl will get into a skip list", 0) \ -DECLARE_SETTINGS_TRAITS(PostgreSQLReplicaSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) +DECLARE_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) -struct PostgreSQLReplicaSettings : public BaseSettings +struct MaterializePostgreSQLSettings : public BaseSettings { void loadFromQuery(ASTStorage & storage_def); }; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index ee15c1ec13d..9e7364c9bb6 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -3,7 +3,7 @@ #if USE_LIBPQXX #include #include -#include +#include #include #include @@ -27,7 +27,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( bool allow_minimal_ddl_, bool is_postgresql_replica_database_engine_, const String tables_list_) - : log(&Poco::Logger::get("PostgreSQLReplicaHandler")) + : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , context(context_) , database_name(database_name_) , connection_str(conn_str) @@ -49,7 +49,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } -void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StoragePostgreSQLReplica * storage) +void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage) { storages[table_name] = storage; } @@ -138,7 +138,7 @@ void PostgreSQLReplicationHandler::startSynchronization() ntx->commit(); - consumer = std::make_shared( + consumer = std::make_shared( context, connection, replication_slot, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 7b9605be5dc..12fbd782887 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -6,8 +6,8 @@ #if USE_LIBPQXX #include "PostgreSQLConnection.h" -#include "PostgreSQLReplicaConsumer.h" -#include "PostgreSQLReplicaMetadata.h" +#include "MaterializePostgreSQLConsumer.h" +#include "MaterializePostgreSQLMetadata.h" #include @@ -19,7 +19,7 @@ namespace DB /// exist in CH, it can be loaded via snapshot while stream is stopped and then comparing wal positions with /// current lsn and table start lsn. -class StoragePostgreSQLReplica; +class StorageMaterializePostgreSQL; class PostgreSQLReplicationHandler { @@ -41,7 +41,7 @@ public: void shutdownFinal(); - void addStorage(const std::string & table_name, StoragePostgreSQLReplica * storage); + void addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage); NameSet fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_); @@ -49,7 +49,7 @@ public: private: using NontransactionPtr = std::shared_ptr; - using Storages = std::unordered_map; + using Storages = std::unordered_map; bool isPublicationExist(std::shared_ptr tx); @@ -83,7 +83,7 @@ private: std::string tables_list, replication_slot, publication_name; PostgreSQLConnectionPtr connection; - std::shared_ptr consumer; + std::shared_ptr consumer; BackgroundSchedulePool::TaskHolder startup_task, consumer_task; std::atomic tables_loaded = false, stop_synchronization = false; @@ -96,4 +96,3 @@ private: } #endif - diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp similarity index 87% rename from src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp rename to src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index dbacc995c67..f239b9e78d4 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -1,4 +1,4 @@ -#include "StoragePostgreSQLReplica.h" +#include "StorageMaterializePostgreSQL.h" #if USE_LIBPQXX #include @@ -36,20 +36,20 @@ namespace ErrorCodes static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree"; -StoragePostgreSQLReplica::StoragePostgreSQLReplica( +StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, const String & remote_database_name, const String & remote_table_name_, const String & connection_str, const StorageInMemoryMetadata & storage_metadata, const Context & context_, - std::unique_ptr replication_settings_) + std::unique_ptr replication_settings_) : IStorage(table_id_) , remote_table_name(remote_table_name_) , global_context(std::make_shared(context_.getGlobalContext())) , replication_settings(std::move(replication_settings_)) , is_postgresql_replica_database( - DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "PostgreSQLReplica") + DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") { setInMemoryMetadata(storage_metadata); @@ -68,7 +68,7 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( } -StoragePostgreSQLReplica::StoragePostgreSQLReplica( +StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, StoragePtr nested_storage_, const Context & context_) @@ -76,13 +76,13 @@ StoragePostgreSQLReplica::StoragePostgreSQLReplica( , global_context(std::make_shared(context_)) , nested_storage(nested_storage_) , is_postgresql_replica_database( - DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "PostgreSQLReplica") + DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") { } -std::string StoragePostgreSQLReplica::getNestedTableName() const +std::string StorageMaterializePostgreSQL::getNestedTableName() const { auto table_name = getStorageID().table_name; @@ -93,7 +93,7 @@ std::string StoragePostgreSQLReplica::getNestedTableName() const } -std::shared_ptr StoragePostgreSQLReplica::getMaterializedColumnsDeclaration( +std::shared_ptr StorageMaterializePostgreSQL::getMaterializedColumnsDeclaration( const String name, const String type, UInt64 default_value) { auto column_declaration = std::make_shared(); @@ -111,7 +111,7 @@ std::shared_ptr StoragePostgreSQLReplica::getMaterializedC } -ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_type) const +ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) const { WhichDataType which(data_type); @@ -152,10 +152,10 @@ ASTPtr StoragePostgreSQLReplica::getColumnDeclaration(const DataTypePtr & data_t } -/// For single storage PostgreSQLReplica get columns and primary key columns from storage definition. -/// For database engine PostgreSQLReplica get columns and primary key columns by fetching from PostgreSQL, also using the same +/// For single storage MaterializePostgreSQL get columns and primary key columns from storage definition. +/// For database engine MaterializePostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same /// transaction with snapshot, which is used for initial tables dump. -ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function & fetch_table_structure) +ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::function & fetch_table_structure) { auto create_table_query = std::make_shared(); @@ -240,7 +240,7 @@ ASTPtr StoragePostgreSQLReplica::getCreateNestedTableQuery(const std::function

& fetch_table_structure) +void StorageMaterializePostgreSQL::createNestedIfNeeded(const std::function & fetch_table_structure) { if (nested_loaded) { @@ -267,7 +267,7 @@ void StoragePostgreSQLReplica::createNestedIfNeeded(const std::functionshutdown(); } -void StoragePostgreSQLReplica::shutdownFinal() +void StorageMaterializePostgreSQL::shutdownFinal() { if (is_postgresql_replica_database) return; @@ -333,7 +333,7 @@ void StoragePostgreSQLReplica::shutdownFinal() } -void StoragePostgreSQLReplica::dropNested() +void StorageMaterializePostgreSQL::dropNested() { std::lock_guard lock(nested_mutex); nested_loaded = false; @@ -351,11 +351,11 @@ void StoragePostgreSQLReplica::dropNested() interpreter.execute(); nested_storage = nullptr; - LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Dropped (or temporarily) nested table {}", getNestedTableName()); + LOG_WARNING(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Dropped (or temporarily) nested table {}", getNestedTableName()); } -NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const +NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const { if (nested_storage) return nested_storage->getVirtuals(); @@ -364,7 +364,7 @@ NamesAndTypesList StoragePostgreSQLReplica::getVirtuals() const } -Pipe StoragePostgreSQLReplica::read( +Pipe StorageMaterializePostgreSQL::read( const Names & column_names, const StorageMetadataPtr & /* metadata_snapshot */, SelectQueryInfo & query_info, @@ -442,24 +442,24 @@ Pipe StoragePostgreSQLReplica::read( return pipe; } - LOG_WARNING(&Poco::Logger::get("StoragePostgreSQLReplica"), "Nested table {} is unavailable or is not loaded yet", getNestedTableName()); + LOG_WARNING(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Nested table {} is unavailable or is not loaded yet", getNestedTableName()); return Pipe(); } -void registerStoragePostgreSQLReplica(StorageFactory & factory) +void registerStorageMaterializePostgreSQL(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; bool has_settings = args.storage_def->settings; - auto postgresql_replication_settings = std::make_unique(); + auto postgresql_replication_settings = std::make_unique(); if (has_settings) postgresql_replication_settings->loadFromQuery(*args.storage_def); if (engine_args.size() != 5) - throw Exception("Storage PostgreSQLReplica requires 5 parameters: " + throw Exception("Storage MaterializePostgreSQL requires 5 parameters: " "PostgreSQL('host:port', 'database', 'table', 'username', 'password'", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -474,7 +474,7 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); if (!args.storage_def->order_by) - throw Exception("Storage PostgreSQLReplica needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Storage MaterializePostgreSQL needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS); if (args.storage_def->primary_key) metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); @@ -493,14 +493,14 @@ void registerStoragePostgreSQLReplica(StorageFactory & factory) engine_args[3]->as().value.safeGet(), engine_args[4]->as().value.safeGet()); - return StoragePostgreSQLReplica::create( + return StorageMaterializePostgreSQL::create( args.table_id, remote_database, remote_table, connection.conn_str(), metadata, args.context, std::move(postgresql_replication_settings)); }; factory.registerStorage( - "PostgreSQLReplica", + "MaterializePostgreSQL", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, diff --git a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h similarity index 83% rename from src/Storages/PostgreSQL/StoragePostgreSQLReplica.h rename to src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 4d407f337ad..544bae1bf85 100644 --- a/src/Storages/PostgreSQL/StoragePostgreSQLReplica.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -6,7 +6,7 @@ #if USE_LIBPQXX #include "PostgreSQLReplicationHandler.h" -#include "PostgreSQLReplicaSettings.h" +#include "MaterializePostgreSQLSettings.h" #include #include @@ -24,17 +24,17 @@ namespace DB { -class StoragePostgreSQLReplica final : public ext::shared_ptr_helper, public IStorage +class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage { - friend struct ext::shared_ptr_helper; + friend struct ext::shared_ptr_helper; public: - StoragePostgreSQLReplica( + StorageMaterializePostgreSQL( const StorageID & table_id_, StoragePtr nested_storage_, const Context & context_); - String getName() const override { return "PostgreSQLReplica"; } + String getName() const override { return "MaterializePostgreSQL"; } void startup() override; void shutdown() override; @@ -70,14 +70,14 @@ public: void dropNested(); protected: - StoragePostgreSQLReplica( + StorageMaterializePostgreSQL( const StorageID & table_id_, const String & remote_database_name, const String & remote_table_name, const String & connection_str, const StorageInMemoryMetadata & storage_metadata, const Context & context_, - std::unique_ptr replication_settings_); + std::unique_ptr replication_settings_); private: static std::shared_ptr getMaterializedColumnsDeclaration( @@ -92,7 +92,7 @@ private: std::string remote_table_name; std::shared_ptr global_context; - std::unique_ptr replication_settings; + std::unique_ptr replication_settings; std::unique_ptr replication_handler; std::atomic nested_loaded = false; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 0f5a3acaa86..bd32de1c315 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -60,7 +60,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory); #if USE_LIBPQXX void registerStoragePostgreSQL(StorageFactory & factory); -void registerStoragePostgreSQLReplica(StorageFactory & factory); +void registerStorageMaterializePostgreSQL(StorageFactory & factory); #endif void registerStorages() @@ -118,7 +118,7 @@ void registerStorages() #if USE_LIBPQXX registerStoragePostgreSQL(factory); - registerStoragePostgreSQLReplica(factory); + registerStorageMaterializePostgreSQL(factory); #endif } From e722cee3750a77b73519e2614b722ee2c4492aa8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Mar 2021 10:52:56 +0000 Subject: [PATCH 036/105] Better templates --- .../PostgreSQLBlockInputStream.cpp | 28 ++++++------------- .../PostgreSQLDictionarySource.cpp | 15 ++++++---- src/Dictionaries/PostgreSQLDictionarySource.h | 1 + 3 files changed, 19 insertions(+), 25 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 7e9aa40e904..495d08fc831 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -23,25 +23,9 @@ namespace DB { -template<> -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( - std::shared_ptr tx_, - const std::string & query_str_, - const Block & sample_block, - const UInt64 max_block_size_, - bool auto_commit_) - : query_str(query_str_) - , max_block_size(max_block_size_) - , auto_commit(auto_commit_) - , tx(tx_) -{ - description.init(sample_block); -} - - -template<> -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( - std::shared_ptr tx_, +template +PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( + std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_, @@ -138,6 +122,12 @@ void PostgreSQLBlockInputStream::readSuffix() } } +template +class PostgreSQLBlockInputStream; + +template +class PostgreSQLBlockInputStream; + } #endif diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 954e5f4e187..5c660f1a5f5 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -68,8 +68,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); - auto tx = std::make_shared(*connection->conn()); - return std::make_shared>(tx, load_all_query, sample_block, max_block_size); + return loadBase(load_all_query); } @@ -77,21 +76,25 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); - auto tx = std::make_shared(*connection->conn()); - return std::make_shared>(tx, load_update_query, sample_block, max_block_size); + return loadBase(load_update_query); } BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); - auto tx = std::make_shared(*connection->conn()); - return std::make_shared>(tx, query, sample_block, max_block_size); + return loadBase(query); } BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); + return loadBase(query); +} + + +BlockInputStreamPtr PostgreSQLDictionarySource::loadBase(const String & query) +{ auto tx = std::make_shared(*connection->conn()); return std::make_shared>(tx, query, sample_block, max_block_size); } diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index a826ff15f4f..f3d6fc15e05 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -37,6 +37,7 @@ public: BlockInputStreamPtr loadUpdatedAll() override; BlockInputStreamPtr loadIds(const std::vector & ids) override; BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + BlockInputStreamPtr loadBase(const String & query); bool isModified() const override; bool supportsSelectiveLoad() const override; From 2dc006acf11eb1335122a537c201e562393e24b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Mar 2021 12:35:02 +0000 Subject: [PATCH 037/105] Rename part 2 --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- src/Interpreters/InterpreterDropQuery.cpp | 2 +- .../test.py | 12 ++++++------ .../test_storage_postgresql_replica/test.py | 18 +++++++++--------- 4 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f3e1dc23447..ee918d29281 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -215,9 +215,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "PostgreSQLReplica" && !context.getSettingsRef().allow_experimental_database_postgresql_replica && !internal) + if (create.storage->engine->name == "MaterializePostgreSQL" && !context.getSettingsRef().allow_experimental_database_postgresql_replica && !internal) { - throw Exception("PostgreSQLReplica is an experimental database engine. " + throw Exception("MaterializePostgreSQL is an experimental database engine. " "Enable allow_experimental_database_postgresql_replica to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 1a282b6ef5d..d2a43629ce4 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -191,7 +191,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat table->shutdown(); #if USE_LIBPQXX - if (table->getName() == "PostgreSQLReplica") + if (table->getName() == "MaterializePostgreSQL") table->as()->shutdownFinal(); #endif diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 9d5686175ab..8611b228392 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -100,7 +100,7 @@ def test_load_and_sync_all_database_tables(started_cluster): create_postgres_table(cursor, table_name); instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(50)".format(table_name)) - instance.query("CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + instance.query("CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") assert 'test_database' in instance.query('SHOW DATABASES') for i in range(NUM_TABLES): @@ -127,7 +127,7 @@ def test_replicating_dml(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i)) instance.query( - "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") for i in range(NUM_TABLES): instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format(i, i)) @@ -188,7 +188,7 @@ def test_different_data_types(started_cluster): )''') instance.query( - "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") for i in range(10): instance.query(''' @@ -262,7 +262,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query(''' CREATE DATABASE test_database - ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') SETTINGS postgresql_replica_tables_list = '{}'; '''.format(publication_tables)) assert 'test_database' in instance.query('SHOW DATABASES') @@ -308,7 +308,7 @@ def test_table_schema_changes(started_cluster): instance.query( """CREATE DATABASE test_database - ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') SETTINGS postgresql_replica_allow_minimal_ddl = 1; """) @@ -354,7 +354,7 @@ def test_changing_replica_identity_value(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)") instance.query( - "CREATE DATABASE test_database ENGINE = PostgreSQLReplica('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)") check_tables_are_synchronized('postgresql_replica'); diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 2ca07092607..4a7a6592873 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -79,7 +79,7 @@ def test_initial_load_from_snapshot(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -103,7 +103,7 @@ def test_no_connection_at_startup(started_cluster): started_cluster.pause_container('postgres1') instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -129,7 +129,7 @@ def test_detach_attach_is_ok(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -164,7 +164,7 @@ def test_replicating_insert_queries(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -205,7 +205,7 @@ def test_replicating_delete_queries(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -243,7 +243,7 @@ def test_replicating_update_queries(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -273,7 +273,7 @@ def test_resume_from_written_version(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -319,7 +319,7 @@ def test_many_replication_messages(started_cluster): _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1, PRIMARY KEY(key)) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') SETTINGS postgresql_replica_max_block_size = 50000; ''') @@ -373,7 +373,7 @@ def test_connection_loss(started_cluster): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) - ENGINE = PostgreSQLReplica( + ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') From 6bb81630e5a9c37a31f9b791bec859a4b776454d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 10:32:11 +0000 Subject: [PATCH 038/105] Small fixes --- .../DatabaseMaterializePostgreSQL.cpp | 43 +++++++------------ .../DatabaseMaterializePostgreSQL.h | 7 +-- .../fetchPostgreSQLTableStructure.cpp | 3 +- .../MaterializePostgreSQLSettings.h | 2 +- .../StorageMaterializePostgreSQL.cpp | 4 +- .../TableFunctionPostgreSQL.cpp | 1 - 6 files changed, 21 insertions(+), 39 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 62c00dfd4a2..98ddb856a31 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -27,6 +27,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; template<> @@ -83,9 +88,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() connection->conn_str(), metadata_path + METADATA_SUFFIX, std::make_shared(global_context), - settings->postgresql_replica_max_block_size.changed - ? settings->postgresql_replica_max_block_size.value - : (global_context.getSettingsRef().max_insert_block_size.value), + settings->postgresql_replica_max_block_size.value, settings->postgresql_replica_allow_minimal_ddl, true, settings->postgresql_replica_tables_list.value); @@ -93,13 +96,15 @@ void DatabaseMaterializePostgreSQL::startSynchronization() for (const auto & table_name : tables_to_replicate) { - auto storage = getStorage(table_name); + auto storage = tryGetTable(table_name, global_context); - if (storage) + if (!storage) { - replication_handler->addStorage(table_name, storage->template as()); - tables[table_name] = storage; + storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), StoragePtr{}, global_context); } + + replication_handler->addStorage(table_name, storage->template as()); + tables[table_name] = storage; } LOG_TRACE(log, "Loaded {} tables. Starting synchronization", tables.size()); @@ -107,18 +112,6 @@ void DatabaseMaterializePostgreSQL::startSynchronization() } -template -StoragePtr DatabaseMaterializePostgreSQL::getStorage(const String & name) -{ - auto storage = tryGetTable(name, global_context); - - if (storage) - return storage; - - return StorageMaterializePostgreSQL::create(StorageID(database_name, name), StoragePtr{}, global_context); -} - - template void DatabaseMaterializePostgreSQL::shutdown() { @@ -151,6 +144,8 @@ void DatabaseMaterializePostgreSQL::loadStoredObjects( template StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const Context & context) const { + /// When a nested ReplacingMergeTree table is managed from PostgreSQLReplicationHandler, its context is modified + /// to show the type of managed table. if (context.hasQueryContext()) { auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; @@ -183,14 +178,8 @@ void DatabaseMaterializePostgreSQL::createTable(const Context & context, c } } - LOG_WARNING(log, "Create table query allowed only for ReplacingMergeTree engine and from synchronization thread"); -} - - -template -void DatabaseMaterializePostgreSQL::dropTable(const Context & context, const String & name, bool no_delay) -{ - Base::dropTable(context, name, no_delay); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Create table query allowed only for ReplacingMergeTree engine and from synchronization thread"); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index fd6d5982fdf..b6a1e70169b 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -52,8 +52,6 @@ public: void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; - void dropTable(const Context & context, const String & name, bool no_delay) override; - void drop(const Context & context) override; void shutdown() override; @@ -61,12 +59,11 @@ public: private: void startSynchronization(); - StoragePtr getStorage(const String & name); - Poco::Logger * log; - const Context global_context; + const Context & global_context; String metadata_path; ASTPtr database_engine_define; + String database_name, remote_database_name; PostgreSQLConnectionPtr connection; std::unique_ptr settings; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 59f0ca62d75..e139914fcc8 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -141,8 +141,7 @@ std::shared_ptr readNamesAndTypesList( } catch (const pqxx::undefined_table &) { - throw Exception(fmt::format( - "PostgreSQL table {} does not exist", postgres_table_name), ErrorCodes::UNKNOWN_TABLE); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", postgres_table_name); } catch (Exception & e) { diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h index 0df618f513e..a2ad76ceaeb 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h @@ -14,7 +14,7 @@ namespace DB #define LIST_OF_POSTGRESQL_REPLICA_SETTINGS(M) \ - M(UInt64, postgresql_replica_max_block_size, 0, "Number of row collected before flushing data into table.", 0) \ + M(UInt64, postgresql_replica_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \ M(String, postgresql_replica_tables_list, "", "List of tables for MaterializePostgreSQL database engine", 0) \ M(Bool, postgresql_replica_allow_minimal_ddl, 0, "Allow to track minimal possible ddl. By default, table after ddl will get into a skip list", 0) \ diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index f239b9e78d4..c1ba5b66679 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -61,9 +61,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( connection_str, metadata_path, global_context, - replication_settings->postgresql_replica_max_block_size.changed - ? replication_settings->postgresql_replica_max_block_size.value - : global_context->getSettingsRef().max_insert_block_size.value, + replication_settings->postgresql_replica_max_block_size.value, replication_settings->postgresql_replica_allow_minimal_ddl.value, false); } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index d243594b726..ba9a3778742 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -12,7 +12,6 @@ #include #include #include "registerTableFunctions.h" -#include #include #include From 4e63b8e5dd3c738a6a748287e300f058a71d676a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 20 Mar 2021 13:50:16 +0300 Subject: [PATCH 039/105] Update src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp Co-authored-by: tavplubix --- .../PostgreSQLReplicationHandler.cpp | 23 +++++++------------ 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 9e7364c9bb6..8033fbf3e45 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -217,26 +217,19 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na void PostgreSQLReplicationHandler::consumerFunc() { - auto start_time = std::chrono::steady_clock::now(); std::vector> skipped_tables; - while (!stop_synchronization) - { - bool reschedule = !consumer->consume(skipped_tables); + bool schedule_now = consumer->consume(skipped_tables); - if (!skipped_tables.empty()) - consumer->updateSkipList(reloadFromSnapshot(skipped_tables)); + if (!skipped_tables.empty()) + consumer->updateSkipList(reloadFromSnapshot(skipped_tables)); - if (reschedule) - break; + if (stop_synchronization) + return; - auto end_time = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - if (duration.count() > max_thread_work_duration_ms) - break; - } - - if (!stop_synchronization) + if (schedule_now) + consumer_task->schedule(); + else consumer_task->scheduleAfter(reschedule_ms); } From caffccd73e068ae2aea43a447426aee217694d39 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 11:53:41 +0000 Subject: [PATCH 040/105] Fixes --- src/DataStreams/PostgreSQLBlockInputStream.h | 1 + .../DatabaseMaterializePostgreSQL.cpp | 2 +- .../MaterializePostgreSQLConsumer.cpp | 13 ++--- .../MaterializePostgreSQLConsumer.h | 4 +- .../PostgreSQLReplicationHandler.cpp | 47 +++++++++---------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 5 +- .../StorageMaterializePostgreSQL.cpp | 6 +-- .../PostgreSQL/StorageMaterializePostgreSQL.h | 2 +- 8 files changed, 38 insertions(+), 42 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index a558a46d153..32eaf25fae5 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -20,6 +20,7 @@ namespace DB template class PostgreSQLBlockInputStream : public IBlockInputStream { + public: PostgreSQLBlockInputStream( std::shared_ptr tx_, diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 98ddb856a31..0da021fd3c8 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -87,7 +87,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() remote_database_name, connection->conn_str(), metadata_path + METADATA_SUFFIX, - std::make_shared(global_context), + global_context, settings->postgresql_replica_max_block_size.value, settings->postgresql_replica_allow_minimal_ddl, true, settings->postgresql_replica_tables_list.value); diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 8e7dc81647e..6448d71adf2 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes } MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( - std::shared_ptr context_, + const Context & context_, PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, @@ -358,10 +358,11 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli } case 'C': // Commit { - readInt8(replication_message, pos, size); /// unused flags - readInt64(replication_message, pos, size); /// Int64 commit lsn - readInt64(replication_message, pos, size); /// Int64 transaction end lsn - readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp + constexpr size_t unused_flags_len = 1; + constexpr size_t commit_lsn_len = 8; + constexpr size_t transaction_end_lsn_len = 8; + constexpr size_t transaction_commit_timestamp_len = 8; + pos += unused_flags_len + commit_lsn_len + transaction_end_lsn_len + transaction_commit_timestamp_len; final_lsn = current_lsn; LOG_DEBUG(log, "Commit lsn: {}", getLSNValue(current_lsn)); /// Will be removed @@ -487,7 +488,7 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptrtable_id = storage->getStorageID(); insert->columns = buffer.columnsAST; - auto insert_context(*context); + auto insert_context(context); insert_context.makeQueryContext(); insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 74dd5d67ccf..97bf926d267 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -27,7 +27,7 @@ public: using Storages = std::unordered_map; MaterializePostgreSQLConsumer( - std::shared_ptr context_, + const Context & context_, PostgreSQLConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, @@ -100,7 +100,7 @@ private: } Poco::Logger * log; - std::shared_ptr context; + const Context & context; const std::string replication_slot_name, publication_name; MaterializePostgreSQLMetadata metadata; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 8033fbf3e45..2752587f9bd 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -15,14 +15,13 @@ namespace DB { static const auto reschedule_ms = 500; -static const auto max_thread_work_duration_ms = 60000; PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const std::string & conn_str, const std::string & metadata_path_, - std::shared_ptr context_, + const Context & context_, const size_t max_block_size_, bool allow_minimal_ddl_, bool is_postgresql_replica_database_engine_, @@ -41,11 +40,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( replication_slot = fmt::format("{}_ch_replication_slot", database_name); publication_name = fmt::format("{}_ch_publication", database_name); - startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); - startup_task->deactivate(); - - consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); - consumer_task->deactivate(); + startup_task = context.getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); + consumer_task = context.getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); } @@ -87,6 +83,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() void PostgreSQLReplicationHandler::shutdown() { stop_synchronization.store(true); + startup_task->deactivate(); consumer_task->deactivate(); } @@ -97,18 +94,18 @@ void PostgreSQLReplicationHandler::startSynchronization() auto replication_connection = std::make_shared(fmt::format("{} replication=database", connection->conn_str())); replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); - auto ntx = std::make_shared(*replication_connection->conn()); + auto tx = std::make_shared(*replication_connection->conn()); std::string snapshot_name, start_lsn; auto initial_sync = [&]() { - createReplicationSlot(ntx, start_lsn, snapshot_name); + createReplicationSlot(tx, start_lsn, snapshot_name); loadFromSnapshot(snapshot_name, storages); }; /// Replication slot should be deleted with drop table only and created only once, reused after detach. - if (!isReplicationSlotExist(ntx, replication_slot)) + if (!isReplicationSlotExist(tx, replication_slot)) { initial_sync(); } @@ -117,7 +114,7 @@ void PostgreSQLReplicationHandler::startSynchronization() /// In case of some failure, the following cases are possible (since publication and replication slot are reused): /// 1. If replication slot exists and metadata file (where last synced version is written) does not exist, it is not ok. /// 2. If created a new publication and replication slot existed before it was created, it is not ok. - dropReplicationSlot(ntx); + dropReplicationSlot(tx); initial_sync(); } else @@ -136,7 +133,7 @@ void PostgreSQLReplicationHandler::startSynchronization() } } - ntx->commit(); + tx->commit(); consumer = std::make_shared( context, @@ -151,8 +148,6 @@ void PostgreSQLReplicationHandler::startSynchronization() nested_storages); consumer_task->activateAndSchedule(); - - replication_connection->conn()->close(); } @@ -287,10 +282,10 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded( } -bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name) +bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr tx, std::string & slot_name) { std::string query_str = fmt::format("SELECT active, restart_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); - pqxx::result result{ntx->exec(query_str)}; + pqxx::result result{tx->exec(query_str)}; /// Replication slot does not exist if (result.empty()) @@ -305,7 +300,7 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr ntx, void PostgreSQLReplicationHandler::createReplicationSlot( - NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name, bool temporary) + NontransactionPtr tx, std::string & start_lsn, std::string & snapshot_name, bool temporary) { std::string query_str; @@ -319,7 +314,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot( try { - pqxx::result result{ntx->exec(query_str)}; + pqxx::result result{tx->exec(query_str)}; start_lsn = result[0][1].as(); snapshot_name = result[0][2].as(); LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, start_lsn); @@ -332,7 +327,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot( } -void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, bool temporary) +void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr tx, bool temporary) { std::string slot_name; if (temporary) @@ -342,15 +337,15 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr ntx, bo std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); - ntx->exec(query_str); + tx->exec(query_str); LOG_TRACE(log, "Dropped replication slot: {}", slot_name); } -void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr ntx) +void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr tx) { std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); - ntx->exec(query_str); + tx->exec(query_str); } @@ -400,7 +395,7 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(PostgreSQLConne PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure( std::shared_ptr tx, const std::string & table_name) { - auto use_nulls = context->getSettingsRef().external_table_functions_use_nulls; + auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; return fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true); } @@ -425,12 +420,12 @@ std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapsh auto replication_connection = std::make_shared(fmt::format("{} replication=database", connection_str)); replication_connection->conn()->set_variable("default_transaction_isolation", "'repeatable read'"); - auto ntx = std::make_shared(*replication_connection->conn()); + auto r_tx = std::make_shared(*replication_connection->conn()); std::string snapshot_name, start_lsn; - createReplicationSlot(ntx, start_lsn, snapshot_name, true); + createReplicationSlot(r_tx, start_lsn, snapshot_name, true); /// This snapshot is valid up to the end of the transaction, which exported it. auto success_tables = loadFromSnapshot(snapshot_name, sync_storages); - ntx->commit(); + r_tx->commit(); for (const auto & relation : relation_data) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 12fbd782887..00ff0c3c488 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -24,12 +24,11 @@ class StorageMaterializePostgreSQL; class PostgreSQLReplicationHandler { public: - friend class PGReplicaLSN; PostgreSQLReplicationHandler( const std::string & database_name_, const std::string & conn_str_, const std::string & metadata_path_, - std::shared_ptr context_, + const Context & context_, const size_t max_block_size_, bool allow_minimal_ddl_, bool is_postgresql_replica_database_engine_, @@ -76,7 +75,7 @@ private: std::unordered_map reloadFromSnapshot(const std::vector> & relation_data); Poco::Logger * log; - std::shared_ptr context; + const Context & context; const std::string database_name, connection_str, metadata_path; const size_t max_block_size; bool allow_minimal_ddl, is_postgresql_replica_database_engine; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index c1ba5b66679..14af357a4ab 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -46,7 +46,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( std::unique_ptr replication_settings_) : IStorage(table_id_) , remote_table_name(remote_table_name_) - , global_context(std::make_shared(context_.getGlobalContext())) + , global_context(context_.getGlobalContext()) , replication_settings(std::move(replication_settings_)) , is_postgresql_replica_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") @@ -71,7 +71,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( StoragePtr nested_storage_, const Context & context_) : IStorage(table_id_) - , global_context(std::make_shared(context_)) + , global_context(context_) , nested_storage(nested_storage_) , is_postgresql_replica_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") @@ -267,7 +267,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(const std::function global_context; + const Context global_context; std::unique_ptr replication_settings; std::unique_ptr replication_handler; From 8091c8724dca603f7b40f19688775f94f6163741 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 12:15:32 +0000 Subject: [PATCH 041/105] Fix metadata file --- .../MaterializePostgreSQLMetadata.cpp | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp index 5cc68b429c0..ee945c67c1a 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp @@ -51,14 +51,14 @@ void MaterializePostgreSQLMetadata::writeMetadata(bool append_metadata) { WriteBufferFromFile out(tmp_metadata_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); - if (!append_metadata) + if (append_metadata) { - writeString("\nLast version:\t" + toString(last_version), out); - writeString("\nLast LSN:\t" + toString(last_lsn), out); + writeString("\nActual LSN:\t" + toString(last_lsn), out); } else { - writeString("\nActual LSN:\t" + toString(last_lsn), out); + writeString("\nLast version:\t" + toString(last_version), out); + writeString("\nLast LSN:\t" + toString(last_lsn), out); } out.next(); @@ -78,6 +78,15 @@ void MaterializePostgreSQLMetadata::commitMetadata(std::string & lsn, const std: try { actual_lsn = finalizeStreamFunc(); + + /// This is not supposed to happen + if (actual_lsn != last_lsn) + { + writeMetadata(true); + LOG_WARNING(&Poco::Logger::get("MaterializePostgreSQLMetadata"), + "Last written LSN {} is not equal to actual LSN {}", last_lsn, actual_lsn); + } + Poco::File(tmp_metadata_file).renameTo(metadata_file); } catch (...) @@ -85,14 +94,6 @@ void MaterializePostgreSQLMetadata::commitMetadata(std::string & lsn, const std: Poco::File(tmp_metadata_file).remove(); throw; } - - /// This is not supposed to happen - if (actual_lsn != last_lsn) - { - writeMetadata(true); - LOG_WARNING(&Poco::Logger::get("MaterializePostgreSQLMetadata"), - "Last written LSN {} is not equal to actual LSN {}", last_lsn, actual_lsn); - } } } From 91b79f95bc5cad93b3bbe9edf45034de53a53a38 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 12:24:35 +0000 Subject: [PATCH 042/105] Better templates --- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 15 +++++---------- .../PostgreSQL/fetchPostgreSQLTableStructure.h | 3 ++- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index e139914fcc8..14004021401 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -154,7 +154,7 @@ std::shared_ptr readNamesAndTypesList( template -PostgreSQLTableStructure fetchPostgreSQLTableStructureImpl( +PostgreSQLTableStructure fetchPostgreSQLTableStructure( std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) { PostgreSQLTableStructure table; @@ -194,19 +194,14 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructureImpl( } +template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) -{ - return fetchPostgreSQLTableStructureImpl(tx, postgres_table_name, use_nulls, with_primary_key); -} + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key); -/// For the case when several operations are made on the transaction object before it can be used (like export snapshot and isolation level) +template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) -{ - return fetchPostgreSQLTableStructureImpl(tx, postgres_table_name, use_nulls, with_primary_key); -} + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key); PostgreSQLTableStructure fetchPostgreSQLTableStructure( diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index bf3c8ead422..485f1f1742e 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -23,8 +23,9 @@ struct PostgreSQLTableStructure PostgreSQLTableStructure fetchPostgreSQLTableStructure( PostgreSQLConnection::ConnectionPtr connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); +template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); } From 60fbeb83546bd7ea91641da245144fe24a5d5f6d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 12:53:12 +0000 Subject: [PATCH 043/105] Better replica consumer code --- .../MaterializePostgreSQLConsumer.cpp | 32 +++++++++++-------- .../MaterializePostgreSQLConsumer.h | 4 +-- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 6448d71adf2..50278fe70aa 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -52,7 +52,7 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( } -void MaterializePostgreSQLConsumer::Buffer::fillBuffer(StoragePtr storage) +void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage) { const auto storage_metadata = storage->getInMemoryMetadataPtr(); description.init(storage_metadata->getSampleBlock()); @@ -281,9 +281,7 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli Int8 new_tuple = readInt8(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; auto buffer = buffers.find(table_name); - - if (buffer == buffers.end()) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Buffer for table {} does not exist", table_name); + assert(buffer != buffers.end()); if (new_tuple) readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT); @@ -299,6 +297,7 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli const auto & table_name = relation_id_to_name[relation_id]; auto buffer = buffers.find(table_name); + assert(buffer != buffers.end()); auto proccess_identifier = [&](Int8 identifier) -> bool { @@ -352,6 +351,7 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli const auto & table_name = relation_id_to_name[relation_id]; auto buffer = buffers.find(table_name); + assert(buffer != buffers.end()); readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE); break; @@ -381,6 +381,16 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli if (!isSyncAllowed(relation_id)) return; + if (storages.find(relation_name) == storages.end()) + { + markTableAsSkipped(relation_id, relation_name); + LOG_ERROR(log, "Storage for table {} does not exist, but is included in replication stream", relation_name); + return; + } + + assert(buffers.count(relation_name)); + + /// 'd' - default (primary key if any) /// 'n' - nothing /// 'f' - all columns (set replica identity full) @@ -443,15 +453,6 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli } } - if (storages.find(relation_name) == storages.end()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Storage for table {} does not exist, but is included in replication stream", relation_name); - } - - [[maybe_unused]] auto buffer_iter = buffers.find(relation_name); - assert(buffer_iter != buffers.end()); - tables_to_sync.insert(relation_name); break; @@ -530,6 +531,9 @@ String MaterializePostgreSQLConsumer::advanceLSN(std::shared_ptrsecond; - buffer.fillBuffer(nested_storage); + buffer.createEmptyBuffer(nested_storage); } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 97bf926d267..7cbfb8dd963 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -65,8 +65,8 @@ private: /// Needed for insertPostgreSQLValue() method to parse array std::unordered_map array_info; - Buffer(StoragePtr storage) { fillBuffer(storage); } - void fillBuffer(StoragePtr storage); + Buffer(StoragePtr storage) { createEmptyBuffer(storage); } + void createEmptyBuffer(StoragePtr storage); }; using Buffers = std::unordered_map; From 4c37ff3fbcf0aa53b099029bb9ec637bcfffeecc Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 13:13:30 +0000 Subject: [PATCH 044/105] Less lambdas --- .../fetchPostgreSQLTableStructure.h | 2 ++ .../PostgreSQLReplicationHandler.cpp | 9 ++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 ++-- .../StorageMaterializePostgreSQL.cpp | 20 +++++++++++-------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 4 ++-- 5 files changed, 24 insertions(+), 15 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 485f1f1742e..265f1f33234 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -20,6 +20,8 @@ struct PostgreSQLTableStructure std::shared_ptr primary_key_columns; }; +using PostgreSQLTableStructurePtr = std::unique_ptr; + PostgreSQLTableStructure fetchPostgreSQLTableStructure( PostgreSQLConnection::ConnectionPtr connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 2752587f9bd..62c334e6dda 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -167,7 +167,7 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); - storage_data.second->createNestedIfNeeded([&]() { return fetchTableStructure(tx, table_name); }); + storage_data.second->createNestedIfNeeded(fetchTableStructure(tx, table_name)); auto nested_storage = storage_data.second->getNested(); /// Load from snapshot, which will show table state before creation of replication slot. @@ -392,11 +392,14 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(PostgreSQLConne } -PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure( +PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( std::shared_ptr tx, const std::string & table_name) { + if (!is_postgresql_replica_database_engine) + return nullptr; + auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - return fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true); + return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true)); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 00ff0c3c488..2b2245082f5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -44,8 +44,6 @@ public: NameSet fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_); - PostgreSQLTableStructure fetchTableStructure(std::shared_ptr tx, const std::string & table_name); - private: using NontransactionPtr = std::shared_ptr; using Storages = std::unordered_map; @@ -74,6 +72,8 @@ private: std::unordered_map reloadFromSnapshot(const std::vector> & relation_data); + PostgreSQLTableStructurePtr fetchTableStructure(std::shared_ptr tx, const std::string & table_name); + Poco::Logger * log; const Context & context; const std::string database_name, connection_str, metadata_path; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 14af357a4ab..92ca0b6c2bf 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -153,7 +153,7 @@ ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & da /// For single storage MaterializePostgreSQL get columns and primary key columns from storage definition. /// For database engine MaterializePostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same /// transaction with snapshot, which is used for initial tables dump. -ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::function & fetch_table_structure) +ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure) { auto create_table_query = std::make_shared(); @@ -175,9 +175,13 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::functi } else { - auto table_structure = fetch_table_structure(); + if (!table_structure) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No table structure returned for table {}.{}", table_id.database_name, table_id.table_name); + } - if (!table_structure.columns) + if (!table_structure->columns) { throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns returned for table {}.{}", table_id.database_name, table_id.table_name); @@ -185,17 +189,17 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::functi StorageInMemoryMetadata storage_metadata; - ordinary_columns_and_types = *table_structure.columns; + ordinary_columns_and_types = *table_structure->columns; storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); setInMemoryMetadata(storage_metadata); - if (!table_structure.primary_key_columns) + if (!table_structure->primary_key_columns) { throw Exception(ErrorCodes::LOGICAL_ERROR, "No primary key columns returned for table {}.{}", table_id.database_name, table_id.table_name); } - auto primary_key_columns = *table_structure.primary_key_columns; + auto primary_key_columns = *table_structure->primary_key_columns; order_by_expression->name = "tuple"; order_by_expression->arguments = std::make_shared(); @@ -238,7 +242,7 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::functi } -void StorageMaterializePostgreSQL::createNestedIfNeeded(const std::function & fetch_table_structure) +void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) { if (nested_loaded) { @@ -249,7 +253,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(const std::function & fetch_table_structure); + void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure); /// Can be nullptr StoragePtr tryGetNested(); @@ -85,7 +85,7 @@ private: ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; - ASTPtr getCreateNestedTableQuery(const std::function & fetch_table_structure); + ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure); std::string getNestedTableName() const; From 8062a81ddd111386795b585919b7337f58cd9664 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 14:13:57 +0000 Subject: [PATCH 045/105] Separate common for materialize* storages --- .../StorageMaterializePostgreSQL.cpp | 72 +++------------- .../ReadFinalForExternalReplicaStorage.cpp | 86 +++++++++++++++++++ .../ReadFinalForExternalReplicaStorage.h | 28 ++++++ src/Storages/StorageMaterializeMySQL.cpp | 66 +++----------- 4 files changed, 136 insertions(+), 116 deletions(-) create mode 100644 src/Storages/ReadFinalForExternalReplicaStorage.cpp create mode 100644 src/Storages/ReadFinalForExternalReplicaStorage.h diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 92ca0b6c2bf..2f950f9823e 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB @@ -368,7 +369,7 @@ NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const Pipe StorageMaterializePostgreSQL::read( const Names & column_names, - const StorageMetadataPtr & /* metadata_snapshot */, + const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -382,66 +383,15 @@ Pipe StorageMaterializePostgreSQL::read( if (!nested_storage) getNested(); - auto storage_lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - - const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); - Block nested_header = nested_metadata->getSampleBlock(); - - ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); - ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); - - NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - - if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) - { - auto & tables_in_select_query = select_query->tables()->as(); - - if (!tables_in_select_query.children.empty()) - { - auto & tables_element = tables_in_select_query.children[0]->as(); - - if (tables_element.table_expression) - tables_element.table_expression->as().final = true; - } - } - - String filter_column_name; - Names require_columns_name = column_names; - ASTPtr expressions = std::make_shared(); - if (column_names_set.empty() || !column_names_set.count(sign_column.name)) - { - require_columns_name.emplace_back(sign_column.name); - - const auto & sign_column_name = std::make_shared(sign_column.name); - const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); - - expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); - filter_column_name = expressions->children.back()->getColumnName(); - - for (const auto & column_name : column_names) - expressions->children.emplace_back(std::make_shared(column_name)); - } - - Pipe pipe = nested_storage->read( - require_columns_name, - nested_metadata, query_info, context, - processed_stage, max_block_size, num_streams); - - pipe.addTableLock(storage_lock); - - if (!expressions->children.empty() && !pipe.empty()) - { - Block pipe_header = pipe.getHeader(); - auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); - ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, expression_actions, filter_column_name, false); - }); - } - - return pipe; + return readFinalFromNestedStorage( + nested_storage, + column_names, + metadata_snapshot, + query_info, + context, + processed_stage, + max_block_size, + num_streams); } LOG_WARNING(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Nested table {} is unavailable or is not loaded yet", getNestedTableName()); diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp new file mode 100644 index 00000000000..37b95eb5d6a --- /dev/null +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -0,0 +1,86 @@ +#include + +#if USE_MYSQL || USE_LIBPQXX + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +Pipe readFinalFromNestedStorage( + StoragePtr nested_storage, + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned int num_streams) +{ + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); + auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); + + Block nested_header = nested_metadata->getSampleBlock(); + ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); + ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + + if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) + { + auto & tables_in_select_query = select_query->tables()->as(); + + if (!tables_in_select_query.children.empty()) + { + auto & tables_element = tables_in_select_query.children[0]->as(); + + if (tables_element.table_expression) + tables_element.table_expression->as().final = true; + } + } + + String filter_column_name; + Names require_columns_name = column_names; + ASTPtr expressions = std::make_shared(); + if (column_names_set.empty() || !column_names_set.count(sign_column.name)) + { + require_columns_name.emplace_back(sign_column.name); + + const auto & sign_column_name = std::make_shared(sign_column.name); + const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); + + expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); + filter_column_name = expressions->children.back()->getColumnName(); + + for (const auto & column_name : column_names) + expressions->children.emplace_back(std::make_shared(column_name)); + } + + Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); + pipe.addTableLock(lock); + + if (!expressions->children.empty() && !pipe.empty()) + { + Block pipe_header = pipe.getHeader(); + auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, expression_actions, filter_column_name, false); + }); + } + + return pipe; +} +} + +#endif diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.h b/src/Storages/ReadFinalForExternalReplicaStorage.h new file mode 100644 index 00000000000..2062392b22f --- /dev/null +++ b/src/Storages/ReadFinalForExternalReplicaStorage.h @@ -0,0 +1,28 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL || USE_LIBPQXX + +#include +#include + + +namespace DB +{ + +Pipe readFinalFromNestedStorage( + StoragePtr nested_storage, + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned int num_streams); + +} + +#endif diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index e59f1e22958..876c9c41e20 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -23,6 +23,7 @@ #include #include +#include namespace DB { @@ -37,7 +38,7 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora Pipe StorageMaterializeMySQL::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -47,60 +48,15 @@ Pipe StorageMaterializeMySQL::read( /// If the background synchronization thread has exception. rethrowSyncExceptionIfNeed(database); - NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); - - Block nested_header = nested_metadata->getSampleBlock(); - ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); - ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); - - if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) - { - auto & tables_in_select_query = select_query->tables()->as(); - - if (!tables_in_select_query.children.empty()) - { - auto & tables_element = tables_in_select_query.children[0]->as(); - - if (tables_element.table_expression) - tables_element.table_expression->as().final = true; - } - } - - String filter_column_name; - Names require_columns_name = column_names; - ASTPtr expressions = std::make_shared(); - if (column_names_set.empty() || !column_names_set.count(sign_column.name)) - { - require_columns_name.emplace_back(sign_column.name); - - const auto & sign_column_name = std::make_shared(sign_column.name); - const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); - - expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); - filter_column_name = expressions->children.back()->getColumnName(); - - for (const auto & column_name : column_names) - expressions->children.emplace_back(std::make_shared(column_name)); - } - - Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); - pipe.addTableLock(lock); - - if (!expressions->children.empty() && !pipe.empty()) - { - Block pipe_header = pipe.getHeader(); - auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); - ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, expression_actions, filter_column_name, false); - }); - } - - return pipe; + return readFinalFromNestedStorage( + nested_storage, + column_names, + metadata_snapshot, + query_info, + context, + processed_stage, + max_block_size, + num_streams); } NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const From 878292d341694241d3524a60dc24154b392bc44e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 14:26:36 +0000 Subject: [PATCH 046/105] Remove redundant rethrow --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 62c334e6dda..e81ad1e0aff 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -200,8 +200,6 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na catch (Exception & e) { tryLogCurrentException(__PRETTY_FUNCTION__); - e.addMessage("while initial data synchronization for table {}", storage_data.first); - throw; } } From ed158e2b7790c2199957c3571a404bef36beac74 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 14:42:03 +0000 Subject: [PATCH 047/105] Proper setting --- src/Core/Settings.h | 1 + src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2af246b50a7..db87d7334b1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -412,6 +412,7 @@ class IColumn; M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ M(Bool, allow_experimental_database_postgresql_replica, false, "Allow to create database with Engine=PostgreSQLReplica(...).", 0) \ + M(Bool, external_databases_use_nulls, true, "If set to false, external databases will use default values instead of NULLs. (Sopported for PostgreSQL/MaterializePostgreSQL database engine)", 0) \ M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 81394eb6fb3..320f39376e7 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -148,7 +148,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte if (!table_checked && !checkPostgresTable(table_name)) return StoragePtr{}; - auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; + auto use_nulls = context.getSettingsRef().external_databases_use_nulls; auto columns = fetchPostgreSQLTableStructure(connection->conn(), doubleQuoteString(table_name), use_nulls).columns; if (!columns) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index e81ad1e0aff..fa48641424b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -197,7 +197,7 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na if (consumer) consumer->updateNested(table_name, nested_storage); } - catch (Exception & e) + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -396,7 +396,7 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( if (!is_postgresql_replica_database_engine) return nullptr; - auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; + auto use_nulls = context.getSettingsRef().external_databases_use_nulls; return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true)); } From 40e50e6b027e91e765aeac9e285f80f6b8f9da1a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 15:12:46 +0000 Subject: [PATCH 048/105] Add unhexN method --- .../MaterializePostgreSQLConsumer.cpp | 42 +++++++++++-------- .../MaterializePostgreSQLConsumer.h | 2 + 2 files changed, 26 insertions(+), 18 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 50278fe70aa..3f562687148 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -145,13 +145,32 @@ void MaterializePostgreSQLConsumer::readString(const char * message, size_t & po } +template +T MaterializePostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t n) +{ + T result = 0; + for (size_t i = 0; i < n; ++i) + { + if (i) result <<= 8; + result |= UInt32(unhex2(message + pos + 2 * i)); + } + return result; +} + + +Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) +{ + assert(size > pos + 16); + Int64 result = unhexN(message, pos, 8); + pos += 16; + return result; +} + + Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 8); - Int32 result = (UInt32(unhex2(message + pos)) << 24) - | (UInt32(unhex2(message + pos + 2)) << 16) - | (UInt32(unhex2(message + pos + 4)) << 8) - | (UInt32(unhex2(message + pos + 6))); + Int32 result = unhexN(message, pos, 4); pos += 8; return result; } @@ -160,8 +179,7 @@ Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & po Int16 MaterializePostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size > pos + 4); - Int16 result = (UInt32(unhex2(message + pos)) << 8) - | (UInt32(unhex2(message + pos + 2))); + Int16 result = unhexN(message, pos, 2); pos += 4; return result; } @@ -176,18 +194,6 @@ Int8 MaterializePostgreSQLConsumer::readInt8(const char * message, size_t & pos, } -Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) -{ - assert(size > pos + 16); - Int64 result = (UInt64(unhex4(message + pos)) << 48) - | (UInt64(unhex4(message + pos + 4)) << 32) - | (UInt64(unhex4(message + pos + 8)) << 16) - | (UInt64(unhex4(message + pos + 12))); - pos += 16; - return result; -} - - void MaterializePostgreSQLConsumer::readTupleData( Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 7cbfb8dd963..1cc0a1fcb20 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -83,6 +83,8 @@ private: void readTupleData(Buffer & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); + template + static T unhexN(const char * message, size_t pos, size_t n); static void readString(const char * message, size_t & pos, size_t size, String & result); static Int64 readInt64(const char * message, size_t & pos, size_t size); static Int32 readInt32(const char * message, size_t & pos, size_t size); From 2100ca6c55439cfc12074ccb4f19fc23166b39e4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 15:27:13 +0000 Subject: [PATCH 049/105] Try smaller timeouts in tests --- .../test_postgresql_replica_database_engine/test.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 8611b228392..b03fb307247 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -43,6 +43,7 @@ def create_postgres_table(cursor, table_name, replica_identity_full=False, templ cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) +@pytest.mark.timeout(30) def assert_nested_table_is_created(table_name): database_tables = instance.query('SHOW TABLES FROM test_database') while table_name not in database_tables: @@ -51,6 +52,7 @@ def assert_nested_table_is_created(table_name): assert(table_name in database_tables) +@pytest.mark.timeout(30) def check_tables_are_synchronized(table_name, order_by='key'): assert_nested_table_is_created(table_name) @@ -88,7 +90,7 @@ def postgresql_setup_teardown(): instance.query('DROP TABLE IF EXISTS test.postgresql_replica') -@pytest.mark.timeout(320) +@pytest.mark.timeout(120) def test_load_and_sync_all_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -115,7 +117,7 @@ def test_load_and_sync_all_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(320) +@pytest.mark.timeout(120) def test_replicating_dml(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -158,7 +160,7 @@ def test_replicating_dml(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(320) +@pytest.mark.timeout(120) def test_different_data_types(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -242,7 +244,7 @@ def test_different_data_types(started_cluster): assert(result == expected) -@pytest.mark.timeout(320) +@pytest.mark.timeout(120) def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -295,7 +297,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(320) +@pytest.mark.timeout(120) def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) From 99295c050afb40eacb65f95fa31d0982a3f6668a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 15:41:15 +0000 Subject: [PATCH 050/105] Better logging --- src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 2f950f9823e..b01defde74d 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -354,7 +354,7 @@ void StorageMaterializePostgreSQL::dropNested() interpreter.execute(); nested_storage = nullptr; - LOG_WARNING(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Dropped (or temporarily) nested table {}", getNestedTableName()); + LOG_TRACE(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Dropped (possibly temporarily) nested table {}", getNestedTableName()); } From fc1c16c4cdab8109901168967a373515efcc8b5d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Mar 2021 17:47:32 +0000 Subject: [PATCH 051/105] Increase timeout for 1 test --- .../integration/test_postgresql_replica_database_engine/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index b03fb307247..e1c7459de91 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -297,7 +297,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(120) +@pytest.mark.timeout(240) def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) From 8586565ce9931d94398c6ffbf0f7105cebf1a615 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Apr 2021 12:43:02 +0000 Subject: [PATCH 052/105] Move connection-related filed to Core/PostgreSQL --- src/CMakeLists.txt | 1 + .../PostgreSQL/PostgreSQLConnection.cpp | 4 +- .../PostgreSQL/PostgreSQLConnection.h | 0 .../PostgreSQL/PostgreSQLConnectionPool.cpp | 0 .../PostgreSQL/PostgreSQLConnectionPool.h | 0 .../PostgreSQL/PostgreSQLPoolWithFailover.cpp | 0 .../PostgreSQL/PostgreSQLPoolWithFailover.h | 0 .../PostgreSQL/insertPostgreSQLValue.cpp | 0 .../PostgreSQL/insertPostgreSQLValue.h | 0 src/Core/ya.make | 4 ++ src/DataStreams/PostgreSQLBlockInputStream.h | 4 +- src/Databases/DatabaseFactory.cpp | 2 +- .../DatabaseMaterializePostgreSQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 1 - src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- .../fetchPostgreSQLTableStructure.h | 2 +- src/Dictionaries/PostgreSQLDictionarySource.h | 2 +- .../MaterializePostgreSQLConsumer.h | 4 +- .../PostgreSQL/PostgreSQLReplicaConnection.h | 39 ------------------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 +- .../StorageMaterializePostgreSQL.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StoragePostgreSQL.h | 2 +- .../TableFunctionPostgreSQL.cpp | 1 - src/TableFunctions/TableFunctionPostgreSQL.h | 2 +- 25 files changed, 23 insertions(+), 57 deletions(-) rename src/{Storages => Core}/PostgreSQL/PostgreSQLConnection.cpp (96%) rename src/{Storages => Core}/PostgreSQL/PostgreSQLConnection.h (100%) rename src/{Storages => Core}/PostgreSQL/PostgreSQLConnectionPool.cpp (100%) rename src/{Storages => Core}/PostgreSQL/PostgreSQLConnectionPool.h (100%) rename src/{Storages => Core}/PostgreSQL/PostgreSQLPoolWithFailover.cpp (100%) rename src/{Storages => Core}/PostgreSQL/PostgreSQLPoolWithFailover.h (100%) rename src/{Storages => Core}/PostgreSQL/insertPostgreSQLValue.cpp (100%) rename src/{Storages => Core}/PostgreSQL/insertPostgreSQLValue.h (100%) delete mode 100644 src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 295cf4904c3..b76132e9cb4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -86,6 +86,7 @@ if (USE_AMQPCPP) endif() if (USE_LIBPQXX) + add_headers_and_sources(dbms Core/PostgreSQL) add_headers_and_sources(dbms Databases/PostgreSQL) add_headers_and_sources(dbms Storages/PostgreSQL) endif() diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp b/src/Core/PostgreSQL/PostgreSQLConnection.cpp similarity index 96% rename from src/Storages/PostgreSQL/PostgreSQLConnection.cpp rename to src/Core/PostgreSQL/PostgreSQLConnection.cpp index 5353ce2bb4f..0a0dbe19e45 100644 --- a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp +++ b/src/Core/PostgreSQL/PostgreSQLConnection.cpp @@ -1,4 +1,4 @@ -#include +#include "PostgreSQLConnection.h" #if USE_LIBPQXX #include @@ -6,6 +6,7 @@ #include #include + namespace DB { namespace ErrorCodes @@ -14,7 +15,6 @@ namespace ErrorCodes } } - namespace postgres { diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.h b/src/Core/PostgreSQL/PostgreSQLConnection.h similarity index 100% rename from src/Storages/PostgreSQL/PostgreSQLConnection.h rename to src/Core/PostgreSQL/PostgreSQLConnection.h diff --git a/src/Storages/PostgreSQL/PostgreSQLConnectionPool.cpp b/src/Core/PostgreSQL/PostgreSQLConnectionPool.cpp similarity index 100% rename from src/Storages/PostgreSQL/PostgreSQLConnectionPool.cpp rename to src/Core/PostgreSQL/PostgreSQLConnectionPool.cpp diff --git a/src/Storages/PostgreSQL/PostgreSQLConnectionPool.h b/src/Core/PostgreSQL/PostgreSQLConnectionPool.h similarity index 100% rename from src/Storages/PostgreSQL/PostgreSQLConnectionPool.h rename to src/Core/PostgreSQL/PostgreSQLConnectionPool.h diff --git a/src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.cpp b/src/Core/PostgreSQL/PostgreSQLPoolWithFailover.cpp similarity index 100% rename from src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.cpp rename to src/Core/PostgreSQL/PostgreSQLPoolWithFailover.cpp diff --git a/src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.h b/src/Core/PostgreSQL/PostgreSQLPoolWithFailover.h similarity index 100% rename from src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.h rename to src/Core/PostgreSQL/PostgreSQLPoolWithFailover.h diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp similarity index 100% rename from src/Storages/PostgreSQL/insertPostgreSQLValue.cpp rename to src/Core/PostgreSQL/insertPostgreSQLValue.cpp diff --git a/src/Storages/PostgreSQL/insertPostgreSQLValue.h b/src/Core/PostgreSQL/insertPostgreSQLValue.h similarity index 100% rename from src/Storages/PostgreSQL/insertPostgreSQLValue.h rename to src/Core/PostgreSQL/insertPostgreSQLValue.h diff --git a/src/Core/ya.make b/src/Core/ya.make index 004653d060e..890ce20e7b3 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -31,6 +31,10 @@ SRCS( MySQL/PacketsProtocolText.cpp MySQL/PacketsReplication.cpp NamesAndTypes.cpp + PostgreSQL/PostgreSQLConnection.cpp + PostgreSQL/PostgreSQLConnectionPool.cpp + PostgreSQL/PostgreSQLPoolWithFailover.cpp + PostgreSQL/insertPostgreSQLValue.cpp PostgreSQLProtocol.cpp QueryProcessingStage.cpp Settings.cpp diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 248ba5c3318..bd22c2ea028 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -10,8 +10,8 @@ #include #include #include -#include -#include +#include +#include namespace DB diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 2bec6663a54..60af31a46cc 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -39,7 +39,7 @@ #include // Y_IGNORE #include #include -#include +#include #endif namespace DB diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index b39382f4cef..49aeff0c5db 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -2,7 +2,7 @@ #if USE_LIBPQXX -#include +#include #include #include diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 887c200d705..eaa1a931fb3 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -16,7 +16,6 @@ #include #include #include -#include namespace DB diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 966d41a35ce..7224490fa04 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index d69ad55db81..bdfa9e0a01b 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -5,7 +5,7 @@ #endif #if USE_LIBPQXX -#include +#include #include diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index f1520a37a79..59c38eb04ed 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 970402b7ba5..e7bb6d22f7e 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -5,9 +5,9 @@ #endif #if USE_LIBPQXX -#include "PostgreSQLConnection.h" #include "MaterializePostgreSQLMetadata.h" -#include "insertPostgreSQLValue.h" +#include +#include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h deleted file mode 100644 index 1ed442873a2..00000000000 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include -#include "PostgreSQLConnectionPool.h" -#include - - -namespace DB -{ - -class PostgreSQLReplicaConnection -{ - -public: - static constexpr inline auto POSTGRESQL_CONNECTION_DEFAULT_RETRIES_NUM = 5; - - PostgreSQLReplicaConnection( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const size_t num_retries_ = POSTGRESQL_CONNECTION_DEFAULT_RETRIES_NUM); - - PostgreSQLReplicaConnection(const PostgreSQLReplicaConnection & other); - - PostgreSQLConnectionHolderPtr get(); - - -private: - /// Highest priority is 0, the bigger the number in map, the less the priority - using ReplicasByPriority = std::map; - - ReplicasByPriority replicas; - size_t num_retries; - std::mutex mutex; -}; - -using PostgreSQLReplicaConnectionPtr = std::shared_ptr; - -} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index aee1d314035..0aa165bd183 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -5,9 +5,11 @@ #endif #if USE_LIBPQXX -#include "PostgreSQLConnection.h" + #include "MaterializePostgreSQLConsumer.h" #include "MaterializePostgreSQLMetadata.h" + +#include #include diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 6d35ced5e93..e90ada126c0 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -22,7 +22,7 @@ #include #include #include -#include "PostgreSQLConnectionPool.h" +#include namespace DB diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 40347d888e5..c764063552b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -1,7 +1,7 @@ #include "StoragePostgreSQL.h" #if USE_LIBPQXX -#include +#include #include #include diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index ec06a698c1f..8a2bbe7d6f0 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 9c202618b63..a88e0557509 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -12,7 +12,6 @@ #include #include #include "registerTableFunctions.h" -#include #include #include diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index 96d46b4fe5a..18f5d5c0d5d 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -5,7 +5,7 @@ #if USE_LIBPQXX #include -#include +#include namespace DB From b544c9fe9ae50393af25e9b7fee1a548c7f98a3e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Apr 2021 13:08:51 +0000 Subject: [PATCH 053/105] Add default template --- src/DataStreams/PostgreSQLBlockInputStream.h | 2 +- src/Dictionaries/PostgreSQLDictionarySource.cpp | 4 ++-- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 3 --- src/Storages/StoragePostgreSQL.cpp | 2 +- 4 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index bd22c2ea028..52c7aa53679 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -17,7 +17,7 @@ namespace DB { -template +template class PostgreSQLBlockInputStream : public IBlockInputStream { diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 2b300c28d9f..9bef164389b 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -93,7 +93,7 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_col BlockInputStreamPtr PostgreSQLDictionarySource::loadBase(const String & query) { - return std::make_shared>(connection->get(), query, sample_block, max_block_size); + return std::make_shared>(connection->get(), query, sample_block, max_block_size); } @@ -115,7 +115,7 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - PostgreSQLBlockInputStream block_input_stream(connection->get(), request, invalidate_sample_block, 1); + PostgreSQLBlockInputStream<> block_input_stream(connection->get(), request, invalidate_sample_block, 1); return readInvalidateQuery(block_input_stream); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index ac0e026a167..a1851cc3248 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -161,9 +161,6 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na auto tx = std::make_shared(connection->getRef()); const auto & table_name = storage_data.first; - /// Specific isolation level is required to read from snapshot. - ///tx->set_variable("transaction_isolation", "'repeatable read'"); - std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index c764063552b..7bc02421f6b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -89,7 +89,7 @@ Pipe StoragePostgreSQL::read( } return Pipe(std::make_shared( - std::make_shared>(pool->get(), query, sample_block, max_block_size_))); + std::make_shared>(pool->get(), query, sample_block, max_block_size_))); } From 4482a35a3a2dbb903ffa33887a99ea0e7cd24c5d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Apr 2021 14:02:15 +0000 Subject: [PATCH 054/105] Check table in a suggested way --- src/DataStreams/PostgreSQLBlockInputStream.h | 4 ++-- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 14 +++----------- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 52c7aa53679..d62c11ccef7 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -67,14 +67,14 @@ class PostgreSQLTransactionBlockInputStream : public PostgreSQLBlockInputStream< { public: - using Base = PostgreSQLBlockInputStream; + using Base = PostgreSQLBlockInputStream; PostgreSQLTransactionBlockInputStream( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block_, const UInt64 max_block_size_) - : PostgreSQLBlockInputStream(tx_, query_str_, sample_block_, max_block_size_, false) {} + : PostgreSQLBlockInputStream(tx_, query_str_, sample_block_, max_block_size_, false) {} void readPrefix() override { diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 4c7047ebe84..4f669a60529 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include @@ -161,21 +162,12 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( { PostgreSQLTableStructure table; - if (postgres_table_name.find('\'') != std::string::npos - || postgres_table_name.find('\\') != std::string::npos) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", - postgres_table_name); - } - std::string query = fmt::format( "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " "attnotnull AS not_null, attndims AS dims " "FROM pg_attribute " - "WHERE attrelid = '{}'::regclass " - "AND NOT attisdropped AND attnum > 0", postgres_table_name); + "WHERE attrelid = {}::regclass " + "AND NOT attisdropped AND attnum > 0", quoteString(postgres_table_name)); table.columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, false); From 093e53e65a9e0760d3ab9643e2e1bf8af3368fec Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Apr 2021 20:39:56 +0000 Subject: [PATCH 055/105] Fix and test server restart --- .../PostgreSQLReplicationHandler.cpp | 2 ++ .../StorageMaterializePostgreSQL.cpp | 7 ++++ .../PostgreSQL/StorageMaterializePostgreSQL.h | 2 ++ .../test.py | 32 ++++++++++++++++++- .../test_storage_postgresql_replica/test.py | 32 ++++++++++++++++++- 5 files changed, 73 insertions(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index a1851cc3248..1cca362ca35 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -119,11 +119,13 @@ void PostgreSQLReplicationHandler::startSynchronization() } else { + LOG_TRACE(log, "Restoring tables..."); for (const auto & [table_name, storage] : storages) { try { nested_storages[table_name] = storage->getNested(); + storage->setStorageMetadata(); storage->setNestedLoaded(); } catch (...) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index e90ada126c0..8b48622da5f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -152,6 +152,13 @@ ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & da } +void StorageMaterializePostgreSQL::setStorageMetadata() +{ + auto storage_metadata = getNested()->getInMemoryMetadataPtr(); + setInMemoryMetadata(*storage_metadata); +} + + /// For single storage MaterializePostgreSQL get columns and primary key columns from storage definition. /// For database engine MaterializePostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same /// transaction with snapshot, which is used for initial tables dump. diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 9d933e84050..feba216b4c4 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -69,6 +69,8 @@ public: void dropNested(); + void setStorageMetadata(); + protected: StorageMaterializePostgreSQL( const StorageID & table_id_, diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index e1c7459de91..535cb0f6a7d 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -10,7 +10,10 @@ from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], user_configs = ['configs/users.xml'], with_postgres=True) +instance = cluster.add_instance('instance', + main_configs=['configs/log_conf.xml'], + user_configs = ['configs/users.xml'], + with_postgres=True, stay_alive=True) postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( @@ -55,6 +58,7 @@ def assert_nested_table_is_created(table_name): @pytest.mark.timeout(30) def check_tables_are_synchronized(table_name, order_by='key'): assert_nested_table_is_created(table_name) + print("nested ok") expected = instance.query('select * from postgres_database.{} order by {};'.format(table_name, order_by)) result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) @@ -364,6 +368,32 @@ def test_changing_replica_identity_value(started_cluster): check_tables_are_synchronized('postgresql_replica'); +@pytest.mark.timeout(320) +def test_clickhouse_restart(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 5 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i)) + + instance.query("CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + check_tables_are_synchronized(table_name); + + for i in range(NUM_TABLES): + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(50000)".format(i, i)) + + instance.restart_clickhouse() + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 4a7a6592873..bca4f159cf6 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -9,7 +9,7 @@ from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True) +instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True, stay_alive=True) postgres_table_template = """ CREATE TABLE IF NOT EXISTS {} ( @@ -400,6 +400,36 @@ def test_connection_loss(started_cluster): assert(int(result) == 100050) +@pytest.mark.timeout(320) +def test_clickhouse_restart(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + ENGINE = MaterializePostgreSQL( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; ''') + + i = 50 + while i < 100000: + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format(i)) + i += 10000 + + instance.restart_clickhouse() + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while int(result) < 100050: + time.sleep(1) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + cursor.execute('DROP TABLE postgresql_replica;') + print(result) + assert(int(result) == 100050) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From bd207daa746a5acbd3ebf31a83c792f3a3c35ffe Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Apr 2021 22:38:17 +0000 Subject: [PATCH 056/105] Use only atomic database --- src/Databases/DatabaseFactory.cpp | 18 +---- .../DatabaseMaterializePostgreSQL.cpp | 74 +++++-------------- .../DatabaseMaterializePostgreSQL.h | 12 +-- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../StorageMaterializePostgreSQL.cpp | 3 +- .../test.py | 2 - 6 files changed, 29 insertions(+), 82 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 60af31a46cc..6b9f90c5500 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -294,20 +294,10 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_define->settings) postgresql_replica_settings->loadFromQuery(*engine_define); - if (create.uuid == UUIDHelpers::Nil) - { - return std::make_shared>( - context, metadata_path, uuid, engine_define, - database_name, postgres_database_name, connection_string, - std::move(postgresql_replica_settings)); - } - else - { - return std::make_shared>( - context, metadata_path, uuid, engine_define, - database_name, postgres_database_name, connection_string, - std::move(postgresql_replica_settings)); - } + return std::make_shared( + context, metadata_path, uuid, engine_define, + database_name, postgres_database_name, connection_string, + std::move(postgresql_replica_settings)); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 49aeff0c5db..dac33e00b3c 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -34,33 +34,7 @@ namespace ErrorCodes static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; -template<> -DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( - const Context & context, - const String & metadata_path_, - UUID /* uuid */, - const ASTStorage * database_engine_define_, - const String & database_name_, - const String & postgres_database_name, - const String & connection_string, - std::unique_ptr settings_) - : DatabaseOrdinary( - database_name_, metadata_path_, "data/" + escapeForFileName(database_name_) + "/", - "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context) - , log(&Poco::Logger::get("MaterializePostgreSQLDatabaseEngine")) - , global_context(context.getGlobalContext()) - , metadata_path(metadata_path_) - , database_engine_define(database_engine_define_->clone()) - , database_name(database_name_) - , remote_database_name(postgres_database_name) - , connection(std::make_shared(connection_string, "")) - , settings(std::move(settings_)) -{ -} - - -template<> -DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( +DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const Context & context, const String & metadata_path_, UUID uuid, @@ -70,8 +44,6 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const String & connection_string, std::unique_ptr settings_) : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context) - , global_context(context.getGlobalContext()) - , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , remote_database_name(postgres_database_name) , connection(std::make_shared(connection_string, "")) @@ -80,8 +52,7 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( } -template -void DatabaseMaterializePostgreSQL::startSynchronization() +void DatabaseMaterializePostgreSQL::startSynchronization() { replication_handler = std::make_unique( remote_database_name, @@ -104,27 +75,24 @@ void DatabaseMaterializePostgreSQL::startSynchronization() } replication_handler->addStorage(table_name, storage->template as()); - tables[table_name] = storage; + materialized_tables[table_name] = storage; } - LOG_TRACE(log, "Loaded {} tables. Starting synchronization", tables.size()); + LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); replication_handler->startup(); } -template -void DatabaseMaterializePostgreSQL::shutdown() +void DatabaseMaterializePostgreSQL::shutdown() { if (replication_handler) replication_handler->shutdown(); } -template -void DatabaseMaterializePostgreSQL::loadStoredObjects( - Context & context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseMaterializePostgreSQL::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) { - Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach); + DatabaseAtomic::loadStoredObjects(context, has_force_restore_data_flag, force_attach); try { @@ -132,7 +100,7 @@ void DatabaseMaterializePostgreSQL::loadStoredObjects( } catch (...) { - tryLogCurrentException(Base::log, "Cannot load nested database objects for PostgreSQL database engine."); + tryLogCurrentException(log, "Cannot load nested database objects for PostgreSQL database engine."); if (!force_attach) throw; @@ -141,8 +109,7 @@ void DatabaseMaterializePostgreSQL::loadStoredObjects( } -template -StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const Context & context) const +StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const Context & context) const { /// When a nested ReplacingMergeTree table is managed from PostgreSQLReplicationHandler, its context is modified /// to show the type of managed table. @@ -151,29 +118,28 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { - return Base::tryGetTable(name, context); + return DatabaseAtomic::tryGetTable(name, context); } } - auto table = tables.find(name); + auto table = materialized_tables.find(name); /// Here it is possible that nested table is temporarily out of reach, but return storage anyway, /// it will not allow to read if nested is unavailable at the moment - if (table != tables.end()) + if (table != materialized_tables.end()) return table->second; return StoragePtr{}; } -template -void DatabaseMaterializePostgreSQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializePostgreSQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) { if (context.hasQueryContext()) { auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { - Base::createTable(context, name, table, query); + DatabaseAtomic::createTable(context, name, table, query); return; } } @@ -183,8 +149,7 @@ void DatabaseMaterializePostgreSQL::createTable(const Context & context, c } -template -void DatabaseMaterializePostgreSQL::drop(const Context & context) +void DatabaseMaterializePostgreSQL::drop(const Context & context) { if (replication_handler) { @@ -193,21 +158,20 @@ void DatabaseMaterializePostgreSQL::drop(const Context & context) } /// Remove metadata - Poco::File metadata(Base::getMetadataPath() + METADATA_SUFFIX); + Poco::File metadata(getMetadataPath() + METADATA_SUFFIX); if (metadata.exists()) metadata.remove(false); - Base::drop(context); + DatabaseAtomic::drop(context); } -template -DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( +DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( const Context & /* context */, const DatabaseOnDisk::FilterByNameFunction & /* filter_by_table_name */) { Tables nested_tables; - for (const auto & [table_name, storage] : tables) + for (const auto & [table_name, storage] : materialized_tables) { auto nested_storage = storage->template as()->tryGetNested(); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index 67c3d5af012..cf9024494f9 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -24,8 +25,7 @@ class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; -template -class DatabaseMaterializePostgreSQL : public Base +class DatabaseMaterializePostgreSQL : public DatabaseAtomic { public: @@ -59,17 +59,13 @@ public: private: void startSynchronization(); - Poco::Logger * log; - const Context & global_context; - String metadata_path; ASTPtr database_engine_define; - - String database_name, remote_database_name; + String remote_database_name; postgres::ConnectionPtr connection; std::unique_ptr settings; std::shared_ptr replication_handler; - std::map tables; + std::map materialized_tables; }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e56cdf13b12..b20a1992a46 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -150,7 +150,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage)); } - if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated") + if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated" || create.storage->engine->name == "MaterializePostgreSQL") { if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. " diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 8b48622da5f..7002d7d8c99 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -75,8 +75,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( : IStorage(table_id_) , global_context(context_) , nested_storage(nested_storage_) - , is_postgresql_replica_database( - DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") + , is_postgresql_replica_database(true) { } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 535cb0f6a7d..7d23458dda1 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -58,7 +58,6 @@ def assert_nested_table_is_created(table_name): @pytest.mark.timeout(30) def check_tables_are_synchronized(table_name, order_by='key'): assert_nested_table_is_created(table_name) - print("nested ok") expected = instance.query('select * from postgres_database.{} order by {};'.format(table_name, order_by)) result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) @@ -81,7 +80,6 @@ def started_cluster(): instance.query(''' CREATE DATABASE postgres_database ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')''') - yield cluster finally: From 3e41c82df157e65ebe9edbc1b761b79fd6127f8e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Apr 2021 23:38:27 +0000 Subject: [PATCH 057/105] Add mutex to tryGetTable only for materialized table (not nested) --- src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp | 2 ++ src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index dac33e00b3c..6c746855b89 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -122,6 +122,8 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const } } + /// Note: In select query we call MaterializePostgreSQL table and it calls tryGetTable from its nested. + std::lock_guard lock(tables_mutex); auto table = materialized_tables.find(name); /// Here it is possible that nested table is temporarily out of reach, but return storage anyway, /// it will not allow to read if nested is unavailable at the moment diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index cf9024494f9..f3d4535866a 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -66,6 +66,7 @@ private: std::shared_ptr replication_handler; std::map materialized_tables; + mutable std::mutex tables_mutex; }; } From f0be5c6938d9194807899ba642043ed62561b0a5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 9 Apr 2021 08:07:56 +0000 Subject: [PATCH 058/105] Fix style, build, ya check --- src/Core/PostgreSQL/PostgreSQLConnection.cpp | 1 - src/Core/ya.make.in | 2 +- .../PostgreSQL/DatabaseMaterializePostgreSQL.cpp | 6 +++--- .../PostgreSQL/DatabaseMaterializePostgreSQL.h | 10 +++++----- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Core/PostgreSQL/PostgreSQLConnection.cpp b/src/Core/PostgreSQL/PostgreSQLConnection.cpp index 0a0dbe19e45..12f0232f326 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnection.cpp +++ b/src/Core/PostgreSQL/PostgreSQLConnection.cpp @@ -2,7 +2,6 @@ #if USE_LIBPQXX #include -#include #include #include diff --git a/src/Core/ya.make.in b/src/Core/ya.make.in index e1c679ac809..53cdafbf32a 100644 --- a/src/Core/ya.make.in +++ b/src/Core/ya.make.in @@ -10,7 +10,7 @@ PEERDIR( SRCS( - + ) END() diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 6c746855b89..d6a02ca2cc9 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -35,15 +35,15 @@ namespace ErrorCodes static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( - const Context & context, + const Context & context_, const String & metadata_path_, - UUID uuid, + UUID uuid_, const ASTStorage * database_engine_define_, const String & database_name_, const String & postgres_database_name, const String & connection_string, std::unique_ptr settings_) - : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context) + : DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context_) , database_engine_define(database_engine_define_->clone()) , remote_database_name(postgres_database_name) , connection(std::make_shared(connection_string, "")) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index f3d4535866a..b80ff4c5974 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -30,12 +30,12 @@ class DatabaseMaterializePostgreSQL : public DatabaseAtomic public: DatabaseMaterializePostgreSQL( - const Context & context, + const Context & context_, const String & metadata_path_, - UUID uuid, - const ASTStorage * database_engine_define, - const String & dbname_, - const String & postgres_dbname, + UUID uuid_, + const ASTStorage * database_engine_define_, + const String & database_name_, + const String & postgres_database_name, const String & connection_string, std::unique_ptr settings_); From 12f98e8b1185847c5124127f20fd97d094b67324 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 9 Apr 2021 10:22:06 +0000 Subject: [PATCH 059/105] Much better --- src/Core/PostgreSQL/PostgreSQLConnection.cpp | 50 +++++++++--- src/Core/PostgreSQL/PostgreSQLConnection.h | 43 +++++++--- .../PostgreSQL/PostgreSQLConnectionPool.cpp | 25 +----- .../PostgreSQL/PostgreSQLConnectionPool.h | 5 +- src/Databases/DatabaseFactory.cpp | 4 +- .../DatabaseMaterializePostgreSQL.cpp | 6 +- .../DatabaseMaterializePostgreSQL.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 80 ++++++++----------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 57 ++++++++++--- .../StorageMaterializePostgreSQL.cpp | 8 +- .../PostgreSQL/StorageMaterializePostgreSQL.h | 2 +- 11 files changed, 166 insertions(+), 116 deletions(-) diff --git a/src/Core/PostgreSQL/PostgreSQLConnection.cpp b/src/Core/PostgreSQL/PostgreSQLConnection.cpp index 12f0232f326..b6128e909ef 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnection.cpp +++ b/src/Core/PostgreSQL/PostgreSQLConnection.cpp @@ -1,9 +1,8 @@ #include "PostgreSQLConnection.h" #if USE_LIBPQXX -#include -#include #include +#include namespace DB @@ -17,11 +16,41 @@ namespace ErrorCodes namespace postgres { -Connection::Connection( - const String & connection_str_, - const String & address_) - : connection_str(connection_str_) - , address(address_) +ConnectionInfo formatConnectionString( + std::string dbname, std::string host, UInt16 port, std::string user, std::string password) +{ + DB::WriteBufferFromOwnString out; + out << "dbname=" << DB::quote << dbname + << " host=" << DB::quote << host + << " port=" << port + << " user=" << DB::quote << user + << " password=" << DB::quote << password; + return std::make_pair(out.str(), host + ':' + DB::toString(port)); +} + + +ConnectionPtr createReplicationConnection(const ConnectionInfo & connection_info) +{ + auto new_connection_info = std::make_pair( + fmt::format("{} replication=database", connection_info.first), + connection_info.second); + + auto connection = std::make_shared(new_connection_info); + connection->get()->set_variable("default_transaction_isolation", "'repeatable read'"); + + return connection; +} + + +template +std::shared_ptr createTransaction(pqxx::connection & connection) +{ + return std::make_shared(connection); +} + + +Connection::Connection(const ConnectionInfo & connection_info_) + : connection_info(connection_info_) { } @@ -54,8 +83,8 @@ void Connection::connectIfNeeded() { if (!connection || !connection->is_open()) { - LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", getAddress()); - connection = std::make_shared(connection_str); + connection = std::make_shared(connection_info.first); + LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second); } } @@ -70,8 +99,7 @@ bool Connection::tryConnectIfNeeded() { LOG_ERROR( &Poco::Logger::get("PostgreSQLConnection"), - "Unable to setup connection to {}, reason: {}", - getAddress(), pqxx_error.what()); + "Unable to setup connection to {}, reason: {}", connection_info.second, pqxx_error.what()); return false; } catch (...) diff --git a/src/Core/PostgreSQL/PostgreSQLConnection.h b/src/Core/PostgreSQL/PostgreSQLConnection.h index 94bb0635914..dfed426b462 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnection.h +++ b/src/Core/PostgreSQL/PostgreSQLConnection.h @@ -8,6 +8,7 @@ #include // Y_IGNORE #include #include +#include namespace pqxx @@ -20,13 +21,40 @@ namespace pqxx namespace postgres { +class Connection; +using ConnectionPtr = std::shared_ptr; + + +/// Connection string and address without login/password (for error logs) +using ConnectionInfo = std::pair; + +ConnectionInfo formatConnectionString( + std::string dbname, std::string host, UInt16 port, std::string user, std::string password); + +ConnectionPtr createReplicationConnection(const ConnectionInfo & connection_info); + + +template +class Transaction +{ +public: + Transaction(pqxx::connection & connection) : transaction(connection) {} + + ~Transaction() { transaction.commit(); } + + T & getRef() { return transaction; } + + void exec(const String & query) { transaction.exec(query); } + +private: + T transaction; +}; + + class Connection { - public: - Connection( - const String & connection_str_, - const String & address_); + Connection(const ConnectionInfo & connection_info_); Connection(const Connection & other) = delete; @@ -38,20 +66,17 @@ public: bool isConnected() { return tryConnectIfNeeded(); } - const String & getConnectionString() { return connection_str; } + const ConnectionInfo & getConnectionInfo() { return connection_info; } private: void connectIfNeeded(); bool tryConnectIfNeeded(); - const std::string & getAddress() { return address; } - pqxx::ConnectionPtr connection; - std::string connection_str, address; + ConnectionInfo connection_info; }; -using ConnectionPtr = std::shared_ptr; class ConnectionHolder { diff --git a/src/Core/PostgreSQL/PostgreSQLConnectionPool.cpp b/src/Core/PostgreSQL/PostgreSQLConnectionPool.cpp index 42c716dcf14..f4a1c7f08f2 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnectionPool.cpp +++ b/src/Core/PostgreSQL/PostgreSQLConnectionPool.cpp @@ -3,8 +3,6 @@ #endif #if USE_LIBPQXX -#include -#include #include "PostgreSQLConnectionPool.h" #include "PostgreSQLConnection.h" #include @@ -31,16 +29,14 @@ ConnectionPool::ConnectionPool( "New connection pool. Size: {}, blocks on empty pool: {}", pool_size, block_on_empty_pool); - address = host + ':' + std::to_string(port); - connection_str = formatConnectionString(std::move(dbname), std::move(host), port, std::move(user), std::move(password)); + connection_info = formatConnectionString(std::move(dbname), std::move(host), port, std::move(user), std::move(password)); initialize(); } ConnectionPool::ConnectionPool(const ConnectionPool & other) : pool(std::make_shared(other.pool_size)) - , connection_str(other.connection_str) - , address(other.address) + , connection_info(other.connection_info) , pool_size(other.pool_size) , pool_wait_timeout(other.pool_wait_timeout) , block_on_empty_pool(other.block_on_empty_pool) @@ -53,20 +49,7 @@ void ConnectionPool::initialize() { /// No connection is made, just fill pool with non-connected connection objects. for (size_t i = 0; i < pool_size; ++i) - pool->push(std::make_shared(connection_str, address)); -} - - -std::string ConnectionPool::formatConnectionString( - std::string dbname, std::string host, UInt16 port, std::string user, std::string password) -{ - DB::WriteBufferFromOwnString out; - out << "dbname=" << DB::quote << dbname - << " host=" << DB::quote << host - << " port=" << port - << " user=" << DB::quote << user - << " password=" << DB::quote << password; - return out.str(); + pool->push(std::make_shared(connection_info)); } @@ -87,7 +70,7 @@ ConnectionHolderPtr ConnectionPool::get() return std::make_shared(connection, *pool); } - connection = std::make_shared(connection_str, address); + connection = std::make_shared(connection_info); return std::make_shared(connection, *pool); } diff --git a/src/Core/PostgreSQL/PostgreSQLConnectionPool.h b/src/Core/PostgreSQL/PostgreSQLConnectionPool.h index b9b2a50aa48..01ae21703d9 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnectionPool.h +++ b/src/Core/PostgreSQL/PostgreSQLConnectionPool.h @@ -41,9 +41,6 @@ public: ConnectionHolderPtr get(); - static std::string formatConnectionString( - std::string dbname, std::string host, UInt16 port, std::string user, std::string password); - private: using Pool = ConcurrentBoundedQueue; using PoolPtr = std::shared_ptr; @@ -51,7 +48,7 @@ private: void initialize(); PoolPtr pool; - std::string connection_str, address; + ConnectionInfo connection_info; size_t pool_size; int64_t pool_wait_timeout; bool block_on_empty_pool; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 6b9f90c5500..06b04a22b9f 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -287,7 +287,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto & password = safeGetLiteralValue(engine_args[3], engine_name); auto parsed_host_port = parseAddress(host_port, 5432); - auto connection_string = postgres::ConnectionPool::formatConnectionString(postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password); + auto connection_info = postgres::formatConnectionString(postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password); auto postgresql_replica_settings = std::make_unique(); @@ -296,7 +296,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared( context, metadata_path, uuid, engine_define, - database_name, postgres_database_name, connection_string, + database_name, postgres_database_name, connection_info, std::move(postgresql_replica_settings)); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index d6a02ca2cc9..7f808007ebc 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -41,12 +41,12 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const ASTStorage * database_engine_define_, const String & database_name_, const String & postgres_database_name, - const String & connection_string, + const postgres::ConnectionInfo & connection_info, std::unique_ptr settings_) : DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context_) , database_engine_define(database_engine_define_->clone()) , remote_database_name(postgres_database_name) - , connection(std::make_shared(connection_string, "")) + , connection(std::make_shared(connection_info)) , settings(std::move(settings_)) { } @@ -56,7 +56,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() { replication_handler = std::make_unique( remote_database_name, - connection->getConnectionString(), + connection->getConnectionInfo(), metadata_path + METADATA_SUFFIX, global_context, settings->postgresql_replica_max_block_size.value, diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index b80ff4c5974..405bfd80283 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -36,7 +36,7 @@ public: const ASTStorage * database_engine_define_, const String & database_name_, const String & postgres_database_name, - const String & connection_string, + const postgres::ConnectionInfo & connection_info, std::unique_ptr settings_); String getEngineName() const override { return "MaterializePostgreSQL"; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 1cca362ca35..cce84892ed6 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -19,7 +19,7 @@ static const auto reschedule_ms = 500; PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, - const std::string & conn_str, + const postgres::ConnectionInfo & connection_info_, const std::string & metadata_path_, const Context & context_, const size_t max_block_size_, @@ -29,13 +29,13 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , context(context_) , database_name(database_name_) - , connection_str(conn_str) , metadata_path(metadata_path_) + , connection_info(connection_info_) , max_block_size(max_block_size_) , allow_minimal_ddl(allow_minimal_ddl_) , is_postgresql_replica_database_engine(is_postgresql_replica_database_engine_) , tables_list(tables_list_) - , connection(std::make_shared(conn_str, "")) + , connection(std::make_shared(connection_info_)) { replication_slot = fmt::format("{}_ch_replication_slot", database_name); publication_name = fmt::format("{}_ch_publication", database_name); @@ -63,14 +63,11 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { /// Will throw pqxx::broken_connection if no connection at the moment connection->get(); - startSynchronization(); } catch (const pqxx::broken_connection & pqxx_error) { - LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", - pqxx_error.what()); - + LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what()); startup_task->scheduleAfter(reschedule_ms); } catch (...) @@ -92,20 +89,19 @@ void PostgreSQLReplicationHandler::startSynchronization() { createPublicationIfNeeded(connection->getRef()); - auto replication_connection = std::make_shared(fmt::format("{} replication=database", connection->getConnectionString()), ""); - replication_connection->get()->set_variable("default_transaction_isolation", "'repeatable read'"); - auto tx = std::make_shared(replication_connection->getRef()); + auto replication_connection = postgres::createReplicationConnection(connection_info); + postgres::Transaction tx(replication_connection->getRef()); std::string snapshot_name, start_lsn; auto initial_sync = [&]() { - createReplicationSlot(tx, start_lsn, snapshot_name); + createReplicationSlot(tx.getRef(), start_lsn, snapshot_name); loadFromSnapshot(snapshot_name, storages); }; /// Replication slot should be deleted with drop table only and created only once, reused after detach. - if (!isReplicationSlotExist(tx, replication_slot)) + if (!isReplicationSlotExist(tx.getRef(), replication_slot)) { initial_sync(); } @@ -114,12 +110,12 @@ void PostgreSQLReplicationHandler::startSynchronization() /// In case of some failure, the following cases are possible (since publication and replication slot are reused): /// 1. If replication slot exists and metadata file (where last synced version is written) does not exist, it is not ok. /// 2. If created a new publication and replication slot existed before it was created, it is not ok. - dropReplicationSlot(tx); + dropReplicationSlot(tx.getRef()); initial_sync(); } else { - LOG_TRACE(log, "Restoring tables..."); + LOG_TRACE(log, "Restoring {} tables...", storages.size()); for (const auto & [table_name, storage] : storages) { try @@ -135,8 +131,6 @@ void PostgreSQLReplicationHandler::startSynchronization() } } - tx->commit(); - consumer = std::make_shared( context, connection, @@ -226,10 +220,10 @@ void PostgreSQLReplicationHandler::consumerFunc() } -bool PostgreSQLReplicationHandler::isPublicationExist(std::shared_ptr tx) +bool PostgreSQLReplicationHandler::isPublicationExist(pqxx::work & tx) { std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name); - pqxx::result result{tx->exec(query_str)}; + pqxx::result result{tx.exec(query_str)}; assert(!result.empty()); bool publication_exists = (result[0][0].as() == "t"); @@ -245,9 +239,9 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::connection & if (new_publication_created) return; - auto tx = std::make_shared(connection_); + postgres::Transaction tx(connection_); - if (!isPublicationExist(tx)) + if (!isPublicationExist(tx.getRef())) { if (tables_list.empty()) { @@ -263,7 +257,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::connection & std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, tables_list); try { - tx->exec(query_str); + tx.exec(query_str); new_publication_created = true; LOG_TRACE(log, "Created publication {} with tables list: {}", publication_name, tables_list); } @@ -273,15 +267,13 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::connection & throw; } } - - tx->commit(); } -bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr tx, std::string & slot_name) +bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & tx, std::string & slot_name) { std::string query_str = fmt::format("SELECT active, restart_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); - pqxx::result result{tx->exec(query_str)}; + pqxx::result result{tx.exec(query_str)}; /// Replication slot does not exist if (result.empty()) @@ -296,7 +288,7 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(NontransactionPtr tx, void PostgreSQLReplicationHandler::createReplicationSlot( - NontransactionPtr tx, std::string & start_lsn, std::string & snapshot_name, bool temporary) + pqxx::nontransaction & tx, std::string & start_lsn, std::string & snapshot_name, bool temporary) { std::string query_str; @@ -310,7 +302,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot( try { - pqxx::result result{tx->exec(query_str)}; + pqxx::result result{tx.exec(query_str)}; start_lsn = result[0][1].as(); snapshot_name = result[0][2].as(); LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, start_lsn); @@ -323,7 +315,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot( } -void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr tx, bool temporary) +void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx, bool temporary) { std::string slot_name; if (temporary) @@ -333,15 +325,15 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(NontransactionPtr tx, boo std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); - tx->exec(query_str); + tx.exec(query_str); LOG_TRACE(log, "Dropped replication slot: {}", slot_name); } -void PostgreSQLReplicationHandler::dropPublication(NontransactionPtr tx) +void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) { std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); - tx->exec(query_str); + tx.exec(query_str); } @@ -350,14 +342,12 @@ void PostgreSQLReplicationHandler::shutdownFinal() if (Poco::File(metadata_path).exists()) Poco::File(metadata_path).remove(); - connection = std::make_shared(connection_str, ""); - auto tx = std::make_shared(connection->getRef()); + connection = std::make_shared(connection_info); + postgres::Transaction tx(connection->getRef()); - dropPublication(tx); - if (isReplicationSlotExist(tx, replication_slot)) - dropReplicationSlot(tx); - - tx->commit(); + dropPublication(tx.getRef()); + if (isReplicationSlotExist(tx.getRef(), replication_slot)) + dropReplicationSlot(tx.getRef()); } @@ -379,9 +369,9 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::connectio { std::string query = fmt::format("SELECT tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); std::unordered_set tables; - pqxx::read_transaction tx(connection_); + postgres::Transaction tx(connection_); - for (auto table_name : tx.stream(query)) + for (auto table_name : tx.getRef().stream(query)) tables.insert(std::get<0>(table_name)); return tables; @@ -405,7 +395,6 @@ std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapsh std::unordered_map tables_start_lsn; try { - auto tx = std::make_shared(connection->getRef()); Storages sync_storages; for (const auto & relation : relation_data) { @@ -414,17 +403,14 @@ std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapsh sync_storages[table_name] = storage; storage->dropNested(); } - tx->commit(); - auto replication_connection = std::make_shared(fmt::format("{} replication=database", connection_str), ""); - replication_connection->get()->set_variable("default_transaction_isolation", "'repeatable read'"); + auto replication_connection = postgres::createReplicationConnection(connection_info); + postgres::Transaction tx(replication_connection->getRef()); - auto r_tx = std::make_shared(replication_connection->getRef()); std::string snapshot_name, start_lsn; - createReplicationSlot(r_tx, start_lsn, snapshot_name, true); + createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, true); /// This snapshot is valid up to the end of the transaction, which exported it. auto success_tables = loadFromSnapshot(snapshot_name, sync_storages); - r_tx->commit(); for (const auto & relation : relation_data) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 0aa165bd183..147c0c7b114 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -28,7 +28,7 @@ class PostgreSQLReplicationHandler public: PostgreSQLReplicationHandler( const std::string & database_name_, - const std::string & conn_str_, + const postgres::ConnectionInfo & connection_info_, const std::string & metadata_path_, const Context & context_, const size_t max_block_size_, @@ -38,29 +38,31 @@ public: void startup(); + /// Stop replication without cleanup. void shutdown(); + /// Clean up replication: remove publication and replication slots. void shutdownFinal(); void addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage); + /// Fetch list of tables which are going to be replicated. Used for database engine. NameSet fetchRequiredTables(pqxx::connection & connection_); private: - using NontransactionPtr = std::shared_ptr; using Storages = std::unordered_map; - bool isPublicationExist(std::shared_ptr tx); - - bool isReplicationSlotExist(NontransactionPtr ntx, std::string & slot_name); - void createPublicationIfNeeded(pqxx::connection & connection_); - void createReplicationSlot(NontransactionPtr ntx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); + bool isPublicationExist(pqxx::work & tx); - void dropReplicationSlot(NontransactionPtr tx, bool temporary = false); + bool isReplicationSlotExist(pqxx::nontransaction & tx, std::string & slot_name); - void dropPublication(NontransactionPtr ntx); + void createReplicationSlot(pqxx::nontransaction & tx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); + + void dropReplicationSlot(pqxx::nontransaction & tx, bool temporary = false); + + void dropPublication(pqxx::nontransaction & ntx); void waitConnectionAndStart(); @@ -78,19 +80,48 @@ private: Poco::Logger * log; const Context & context; - const std::string database_name, connection_str, metadata_path; + + /// Remote database name. + const String database_name; + + /// Path for replication metadata. + const String metadata_path; + + /// Connection string and address for logs. + postgres::ConnectionInfo connection_info; + + /// max_block_size for replication stream. const size_t max_block_size; - bool allow_minimal_ddl, is_postgresql_replica_database_engine; - std::string tables_list, replication_slot, publication_name; + + /// Table structure changes are always tracked. By default, table with changed schema will get into a skip list. + bool allow_minimal_ddl = false; + + /// To distinguish whether current replication handler belongs to a MaterializePostgreSQL database engine or single storage. + bool is_postgresql_replica_database_engine; + + /// A coma-separated list of tables, which are going to be replicated for database engine. By default, a whole database is replicated. + String tables_list; + + String replication_slot, publication_name; postgres::ConnectionPtr connection; + + /// Replication consumer. Manages deconding of replication stream and syncing into tables. std::shared_ptr consumer; BackgroundSchedulePool::TaskHolder startup_task, consumer_task; - std::atomic tables_loaded = false, stop_synchronization = false; + + std::atomic stop_synchronization = false; + + /// For database engine there are 2 places where it is checked for publication: + /// 1. to fetch tables list from already created publication when database is loaded + /// 2. at replication startup bool new_publication_created = false; + /// MaterializePostgreSQL tables. Used for managing all operations with its internal nested tables. Storages storages; + + /// List of nested tables, which is passed to replication consumer. std::unordered_map nested_storages; }; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 7002d7d8c99..65b38086db9 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -42,7 +42,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, const String & remote_database_name, const String & remote_table_name_, - const String & connection_str, + const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, const Context & context_, std::unique_ptr replication_settings_) @@ -60,7 +60,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( replication_handler = std::make_unique( remote_database_name, - connection_str, + connection_info, metadata_path, global_context, replication_settings->postgresql_replica_max_block_size.value, @@ -445,7 +445,7 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) const String & remote_database = engine_args[1]->as().value.safeGet(); /// No connection is made here, see Storages/PostgreSQL/PostgreSQLConnection.cpp - auto connection_string = postgres::ConnectionPool::formatConnectionString( + auto connection_info = postgres::formatConnectionString( remote_database, parsed_host_port.first, parsed_host_port.second, @@ -453,7 +453,7 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) engine_args[4]->as().value.safeGet()); return StorageMaterializePostgreSQL::create( - args.table_id, remote_database, remote_table, connection_string, + args.table_id, remote_database, remote_table, connection_info, metadata, args.context, std::move(postgresql_replication_settings)); }; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index feba216b4c4..5bbea64133a 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -76,7 +76,7 @@ protected: const StorageID & table_id_, const String & remote_database_name, const String & remote_table_name, - const String & connection_str, + const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, const Context & context_, std::unique_ptr replication_settings_); From f7361250b2b19a4bfab880c4789457a166c3a64d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 9 Apr 2021 11:02:12 +0000 Subject: [PATCH 060/105] Fixes --- src/Core/PostgreSQL/PostgreSQLConnection.cpp | 7 ------- src/DataStreams/PostgreSQLBlockInputStream.h | 3 +-- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 5 +++-- src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp | 1 - 4 files changed, 4 insertions(+), 12 deletions(-) diff --git a/src/Core/PostgreSQL/PostgreSQLConnection.cpp b/src/Core/PostgreSQL/PostgreSQLConnection.cpp index b6128e909ef..a6edf3e150e 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnection.cpp +++ b/src/Core/PostgreSQL/PostgreSQLConnection.cpp @@ -42,13 +42,6 @@ ConnectionPtr createReplicationConnection(const ConnectionInfo & connection_info } -template -std::shared_ptr createTransaction(pqxx::connection & connection) -{ - return std::make_shared(connection); -} - - Connection::Connection(const ConnectionInfo & connection_info_) : connection_info(connection_info_) { diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index d62c11ccef7..5c637015f18 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -61,11 +61,10 @@ private: }; -/// Passes transaction object into PostgreSQLBlockInputStream and does not close transaction after read if finished. +/// Passes transaction object into PostgreSQLBlockInputStream and does not close transaction after read is finished. template class PostgreSQLTransactionBlockInputStream : public PostgreSQLBlockInputStream { - public: using Base = PostgreSQLBlockInputStream; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index cce84892ed6..a1825afe622 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -17,6 +17,8 @@ namespace DB static const auto reschedule_ms = 500; +/// TODO: fetch replica identity index + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const postgres::ConnectionInfo & connection_info_, @@ -279,9 +281,8 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & if (result.empty()) return false; - bool is_active = result[0][0].as(); LOG_TRACE(log, "Replication slot {} already exists (active: {}). Restart lsn position is {}", - slot_name, is_active, result[0][0].as()); + slot_name, result[0][0].as(), result[0][0].as()); return true; } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 65b38086db9..a680a5bfd84 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -77,7 +77,6 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , nested_storage(nested_storage_) , is_postgresql_replica_database(true) { - } From dba1fe1989b0ee120485c13ba5769bfa1c9909c6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 9 Apr 2021 14:07:18 +0000 Subject: [PATCH 061/105] Add doc draft --- .../materialize-postgresql.md | 41 +++++++++++++++++++ src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 1 + src/Core/PostgreSQL/insertPostgreSQLValue.h | 1 - .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- 4 files changed, 43 insertions(+), 2 deletions(-) create mode 100644 docs/en/engines/database-engines/materialize-postgresql.md diff --git a/docs/en/engines/database-engines/materialize-postgresql.md b/docs/en/engines/database-engines/materialize-postgresql.md new file mode 100644 index 00000000000..c0572c85bfc --- /dev/null +++ b/docs/en/engines/database-engines/materialize-postgresql.md @@ -0,0 +1,41 @@ +--- +toc_priority: 30 +toc_title: MaterializePostgreSQL +--- + +# MaterializePostgreSQL {#materialize-postgresql} + +## Creating a Database {#creating-a-database} + +## Requirements + +Each replicated table must have one of the following **replica identity**: + +1. **default** (primary key) + +2. **index** + +``` +postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL); +postgres# CREATE unique INDEX postgres_table_index on postgres_table(a, c, e); +postgres# ALTER TABLE postgres_table REPLICA IDENTITY USING INDEX postgres_table_index; + +``` + +3. **full** (all columns, very inefficient) + + +You can check what type is used for a specific table with the following command: + +``` sql +postgres# SELECT CASE relreplident + WHEN 'd' THEN 'default' + WHEN 'n' THEN 'nothing' + WHEN 'f' THEN 'full' + WHEN 'i' THEN 'index' + END AS replica_identity +FROM pg_class +WHERE oid = 'postgres_table'::regclass; + +``` + diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 76e770e8fd0..8d490f253db 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -13,6 +13,7 @@ #include #include #include +#include // Y_IGNORE namespace DB diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.h b/src/Core/PostgreSQL/insertPostgreSQLValue.h index dd093cd4c5b..89d63e44ed3 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.h +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.h @@ -4,7 +4,6 @@ #include #include #include -#include // Y_IGNORE namespace DB diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 147c0c7b114..5557ae63f96 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -106,7 +106,7 @@ private: postgres::ConnectionPtr connection; - /// Replication consumer. Manages deconding of replication stream and syncing into tables. + /// Replication consumer. Manages decoding of replication stream and syncing into tables. std::shared_ptr consumer; BackgroundSchedulePool::TaskHolder startup_task, consumer_task; From 1c501e7d9738bd9f2f4048ea71e047a4a40d7603 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 10 Apr 2021 14:42:45 +0000 Subject: [PATCH 062/105] Support replica identity index --- .../materialize-postgresql.md | 9 ++- src/Core/Settings.h | 2 +- .../fetchPostgreSQLTableStructure.cpp | 57 ++++++++++++++----- .../fetchPostgreSQLTableStructure.h | 6 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../MaterializePostgreSQLConsumer.cpp | 10 +--- .../MaterializePostgreSQLConsumer.h | 3 +- .../PostgreSQLReplicationHandler.cpp | 3 +- .../StorageMaterializePostgreSQL.cpp | 14 +++-- .../configs/users.xml | 2 +- .../test.py | 34 ++++++++++- 11 files changed, 99 insertions(+), 43 deletions(-) diff --git a/docs/en/engines/database-engines/materialize-postgresql.md b/docs/en/engines/database-engines/materialize-postgresql.md index c0572c85bfc..b3516001929 100644 --- a/docs/en/engines/database-engines/materialize-postgresql.md +++ b/docs/en/engines/database-engines/materialize-postgresql.md @@ -15,19 +15,18 @@ Each replicated table must have one of the following **replica identity**: 2. **index** -``` +``` bash postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL); postgres# CREATE unique INDEX postgres_table_index on postgres_table(a, c, e); postgres# ALTER TABLE postgres_table REPLICA IDENTITY USING INDEX postgres_table_index; - ``` -3. **full** (all columns, very inefficient) - +Primary key is always checked first. If it is absent, then index, defined as replica identity index, is checked. +If index is used as replica identity, there has to be only one such index in a table. You can check what type is used for a specific table with the following command: -``` sql +``` bash postgres# SELECT CASE relreplident WHEN 'd' THEN 'default' WHEN 'n' THEN 'nothing' diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2a956adb288..9936d8414b8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -418,7 +418,7 @@ class IColumn; M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ - M(Bool, allow_experimental_database_postgresql_replica, false, "Allow to create database with Engine=PostgreSQLReplica(...).", 0) \ + M(Bool, allow_experimental_database_materialize_postgresql, false, "Allow to create database with Engine=MaterializePostgreSQL(...).", 0) \ M(Bool, external_databases_use_nulls, true, "If set to false, external databases will use default values instead of NULLs. (Sopported for PostgreSQL/MaterializePostgreSQL database engine)", 0) \ M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 4f669a60529..63124ba12d7 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -158,7 +158,7 @@ std::shared_ptr readNamesAndTypesList( template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key) + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key, bool with_replica_identity_index) { PostgreSQLTableStructure table; @@ -171,18 +171,43 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( table.columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, false); - if (!with_primary_key) - return table; + if (with_primary_key) + { + /// wiki.postgresql.org/wiki/Retrieve_primary_key_columns + query = fmt::format( + "SELECT a.attname, format_type(a.atttypid, a.atttypmod) AS data_type " + "FROM pg_index i " + "JOIN pg_attribute a ON a.attrelid = i.indrelid " + "AND a.attnum = ANY(i.indkey) " + "WHERE i.indrelid = '{}'::regclass AND i.indisprimary", postgres_table_name); - /// wiki.postgresql.org/wiki/Retrieve_primary_key_columns - query = fmt::format( - "SELECT a.attname, format_type(a.atttypid, a.atttypmod) AS data_type " - "FROM pg_index i " - "JOIN pg_attribute a ON a.attrelid = i.indrelid " - "AND a.attnum = ANY(i.indkey) " - "WHERE i.indrelid = '{}'::regclass AND i.indisprimary", postgres_table_name); + table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); + } - table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); + if (with_replica_identity_index) + { + query = fmt::format( + "SELECT " + "a.attname AS column_name, " /// column name + "format_type(a.atttypid, a.atttypmod) as type " /// column type + "FROM " + "pg_class t, " + "pg_class i, " + "pg_index ix, " + "pg_attribute a " + "WHERE " + "t.oid = ix.indrelid " + "and i.oid = ix.indexrelid " + "and a.attrelid = t.oid " + "and a.attnum = ANY(ix.indkey) " + "and t.relkind = 'r' " /// simple tables + "and t.relname = '{}' " + "and ix.indisreplident = 't' " /// index is is replica identity index + "ORDER BY a.attname", /// column names + postgres_table_name); + + table.replica_identity_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); + } return table; } @@ -190,19 +215,21 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key); + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, + bool with_primary_key, bool with_replica_identity_index); template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key); + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, + bool with_primary_key, bool with_replica_identity_index); PostgreSQLTableStructure fetchPostgreSQLTableStructure( - pqxx::connection & connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key) + pqxx::connection & connection, const String & postgres_table_name, bool use_nulls) { auto tx = std::make_shared(connection); - auto table = fetchPostgreSQLTableStructure(tx, postgres_table_name, use_nulls, with_primary_key); + auto table = fetchPostgreSQLTableStructure(tx, postgres_table_name, use_nulls, false, false); tx->commit(); return table; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index bdfa9e0a01b..bbcb9cd192f 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -18,16 +18,18 @@ struct PostgreSQLTableStructure { std::shared_ptr columns; std::shared_ptr primary_key_columns; + std::shared_ptr replica_identity_columns; }; using PostgreSQLTableStructurePtr = std::unique_ptr; PostgreSQLTableStructure fetchPostgreSQLTableStructure( - pqxx::connection & connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); + pqxx::connection & connection, const String & postgres_table_name, bool use_nulls); template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false); + std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, + bool with_primary_key = false, bool with_replica_identity_index = false); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b20a1992a46..d3bf61ee612 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -215,7 +215,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "MaterializePostgreSQL" && !context.getSettingsRef().allow_experimental_database_postgresql_replica && !internal) + if (create.storage->engine->name == "MaterializePostgreSQL" && !context.getSettingsRef().allow_experimental_database_materialize_postgresql && !internal) { throw Exception("MaterializePostgreSQL is an experimental database engine. " "Enable allow_experimental_database_postgresql_replica to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 76626c575b7..ed317f03e60 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -31,7 +31,6 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( const std::string & start_lsn, const size_t max_block_size_, bool allow_minimal_ddl_, - bool is_postgresql_replica_database_engine_, Storages storages_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) , context(context_) @@ -42,7 +41,6 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( , current_lsn(start_lsn) , max_block_size(max_block_size_) , allow_minimal_ddl(allow_minimal_ddl_) - , is_postgresql_replica_database_engine(is_postgresql_replica_database_engine_) , storages(storages_) { for (const auto & [table_name, storage] : storages) @@ -401,15 +399,13 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli /// 'n' - nothing /// 'f' - all columns (set replica identity full) /// 'i' - user defined index with indisreplident set - /// For database engine now supported only 'd', for table engine 'f' is also allowed. + /// Only 'd' and 'i' - are supported. char replica_identity = readInt8(replication_message, pos, size); - if (replica_identity != 'd' && (replica_identity != 'f' || is_postgresql_replica_database_engine)) + if (replica_identity != 'd' && replica_identity != 'i') { LOG_WARNING(log, - "Table has replica identity {} - not supported. " - "For database engine only default (with primary keys) replica identity is supported." - "For table engine full replica identity is also supported. Table will be skipped."); + "Table has replica identity {} - not supported. A table must have a primary key or a replica identity index"); markTableAsSkipped(relation_id, relation_name); return; } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index e7bb6d22f7e..a52ecd73e07 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -35,7 +35,6 @@ public: const std::string & start_lsn, const size_t max_block_size_, bool allow_minimal_ddl_, - bool is_postgresql_replica_database_engine_, Storages storages_); void readMetadata(); @@ -110,7 +109,7 @@ private: std::string current_lsn, final_lsn; const size_t max_block_size; - bool allow_minimal_ddl, is_postgresql_replica_database_engine; + bool allow_minimal_ddl; std::string table_to_insert; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index a1825afe622..556ede4ce7b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -142,7 +142,6 @@ void PostgreSQLReplicationHandler::startSynchronization() start_lsn, max_block_size, allow_minimal_ddl, - is_postgresql_replica_database_engine, nested_storages); consumer_task->activateAndSchedule(); @@ -386,7 +385,7 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( return nullptr; auto use_nulls = context.getSettingsRef().external_databases_use_nulls; - return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true)); + return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true, true)); } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index a680a5bfd84..e4b87b8410b 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -200,18 +200,22 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); setInMemoryMetadata(storage_metadata); - if (!table_structure->primary_key_columns) + if (!table_structure->primary_key_columns && !table_structure->replica_identity_columns) { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No primary key columns returned for table {}.{}", table_id.database_name, table_id.table_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table {}.{} has no primary key and no replica identity index", table_id.database_name, table_id.table_name); } - auto primary_key_columns = *table_structure->primary_key_columns; + NamesAndTypesList merging_columns; + if (table_structure->primary_key_columns) + merging_columns = *table_structure->primary_key_columns; + else + merging_columns = *table_structure->replica_identity_columns; order_by_expression->name = "tuple"; order_by_expression->arguments = std::make_shared(); - for (const auto & column : primary_key_columns) + for (const auto & column : merging_columns) order_by_expression->arguments->children.emplace_back(std::make_shared(column.name)); } diff --git a/tests/integration/test_postgresql_replica_database_engine/configs/users.xml b/tests/integration/test_postgresql_replica_database_engine/configs/users.xml index 948093dbf4c..74d2737c821 100644 --- a/tests/integration/test_postgresql_replica_database_engine/configs/users.xml +++ b/tests/integration/test_postgresql_replica_database_engine/configs/users.xml @@ -2,7 +2,7 @@ - 1 + 1 diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 7d23458dda1..cf93a3e1b1c 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -11,7 +11,7 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - main_configs=['configs/log_conf.xml'], + main_configs = ['configs/log_conf.xml'], user_configs = ['configs/users.xml'], with_postgres=True, stay_alive=True) @@ -23,6 +23,10 @@ postgres_table_template_2 = """ CREATE TABLE IF NOT EXISTS {} ( key Integer NOT NULL, value1 Integer, value2 Integer, value3 Integer, PRIMARY KEY(key)) """ +postgres_table_template_3 = """ + CREATE TABLE IF NOT EXISTS {} ( + key1 Integer NOT NULL, value1 Integer, key2 Integer NOT NULL, value2 Integer NOT NULL) + """ def get_postgres_conn(database=False): if database == True: @@ -299,7 +303,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(240) +@pytest.mark.timeout(320) def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) @@ -392,6 +396,32 @@ def test_clickhouse_restart(started_cluster): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); +@pytest.mark.timeout(120) +def test_replica_identity_index(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + + create_postgres_table(cursor, 'postgresql_replica', template=postgres_table_template_3); + 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)") + + instance.query( + "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(100, 10)") + check_tables_are_synchronized('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 ") + check_tables_are_synchronized('postgresql_replica', order_by='key1'); + + cursor.execute('DELETE FROM postgresql_replica WHERE key2<75;') + check_tables_are_synchronized('postgresql_replica', order_by='key1'); + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From bc228f4010cf602df87bba5a34aed8f4bd3f2d00 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 10 Apr 2021 17:58:09 +0000 Subject: [PATCH 063/105] Better way to drop nested table for single storage --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 3 + src/Core/PostgreSQL/insertPostgreSQLValue.h | 8 + src/Databases/DatabaseAtomic.cpp | 11 +- .../DatabaseMaterializePostgreSQL.cpp | 14 +- .../DatabaseMaterializePostgreSQL.h | 2 + .../fetchPostgreSQLTableStructure.cpp | 4 +- src/Interpreters/InterpreterDropQuery.cpp | 40 ++++- src/Interpreters/InterpreterDropQuery.h | 2 + src/Storages/IStorage.h | 2 + .../PostgreSQLReplicationHandler.cpp | 12 +- .../StorageMaterializePostgreSQL.cpp | 147 ++++++------------ .../PostgreSQL/StorageMaterializePostgreSQL.h | 22 +-- src/Storages/StorageMaterializedView.cpp | 39 +---- src/Storages/StorageMaterializedView.h | 2 +- 14 files changed, 135 insertions(+), 173 deletions(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 8d490f253db..70537767dc5 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -1,5 +1,6 @@ #include "insertPostgreSQLValue.h" +#if USE_LIBPQXX #include #include #include @@ -233,3 +234,5 @@ void preparePostgreSQLArrayInfo( array_info[column_idx] = {count_dimensions, default_value, parser}; } } + +#endif diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.h b/src/Core/PostgreSQL/insertPostgreSQLValue.h index 89d63e44ed3..7acba4f09bd 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.h +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.h @@ -1,5 +1,11 @@ #pragma once +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX + #include #include #include @@ -28,3 +34,5 @@ void preparePostgreSQLArrayInfo( void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_column); } + +#endif diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index b4222a7e349..ef1917091e3 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -108,12 +108,10 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) void DatabaseAtomic::dropTable(const Context & context, const String & table_name, bool no_delay) { - if (auto * mv = dynamic_cast(tryGetTable(table_name, context).get())) - { - /// Remove the inner table (if any) to avoid deadlock - /// (due to attempt to execute DROP from the worker thread) - mv->dropInnerTable(no_delay, context); - } + auto * storage = tryGetTable(table_name, context).get(); + /// Remove the inner table (if any) to avoid deadlock + /// (due to attempt to execute DROP from the worker thread) + storage->dropInnerTableIfAny(no_delay, context); String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop; @@ -594,4 +592,3 @@ void DatabaseAtomic::waitDetachedTableNotInUse(const UUID & uuid) } } - diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 7f808007ebc..ad60b6242ff 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -70,9 +70,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() auto storage = tryGetTable(table_name, global_context); if (!storage) - { - storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), StoragePtr{}, global_context); - } + storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), global_context); replication_handler->addStorage(table_name, storage->template as()); materialized_tables[table_name] = storage; @@ -151,13 +149,17 @@ void DatabaseMaterializePostgreSQL::createTable(const Context & context, const S } +void DatabaseMaterializePostgreSQL::stopReplication() +{ + if (replication_handler) + replication_handler->shutdown(); +} + + void DatabaseMaterializePostgreSQL::drop(const Context & context) { if (replication_handler) - { - replication_handler->shutdown(); replication_handler->shutdownFinal(); - } /// Remove metadata Poco::File metadata(getMetadataPath() + METADATA_SUFFIX); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index 405bfd80283..b87d281d7a5 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -56,6 +56,8 @@ public: void shutdown() override; + void stopReplication(); + private: void startSynchronization(); diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 63124ba12d7..01ade1da180 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -184,7 +184,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); } - if (with_replica_identity_index) + if (with_replica_identity_index && !table.primary_key_columns) { query = fmt::format( "SELECT " @@ -201,7 +201,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "and a.attrelid = t.oid " "and a.attnum = ANY(ix.indkey) " "and t.relkind = 'r' " /// simple tables - "and t.relname = '{}' " + "and t.relname = '{}' " /// Connection is alread done to a needed database, only table name is needed. "and ix.indisreplident = 't' " /// index is is replica identity index "ORDER BY a.attname", /// column names postgres_table_name); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 2c698b5e3c1..6417b32e389 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -23,7 +23,7 @@ #endif #if USE_LIBPQXX -# include +# include #endif namespace DB @@ -186,11 +186,6 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP table->shutdown(); -#if USE_LIBPQXX - if (table->getName() == "MaterializePostgreSQL") - table->as()->shutdownFinal(); -#endif - TableExclusiveLockHolder table_lock; if (database->getUUID() == UUIDHelpers::Nil) table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); @@ -353,6 +348,10 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, #endif if (auto * replicated = typeid_cast(database.get())) replicated->stopReplication(); +#if USE_LIBPQXX + if (auto * materialize_postgresql = typeid_cast(database.get())) + materialize_postgresql->stopReplication(); +#endif if (database->shouldBeEmptyOnDetach()) { @@ -434,4 +433,33 @@ void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const elem.query_kind = "Drop"; } +void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, const Context & global_context, const Context & current_context, const StorageID & target_table_id, bool no_delay) +{ + if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) + { + /// We create and execute `drop` query for internal table. + auto drop_query = std::make_shared(); + drop_query->database = target_table_id.database_name; + drop_query->table = target_table_id.table_name; + drop_query->kind = kind; + drop_query->no_delay = no_delay; + drop_query->if_exists = true; + ASTPtr ast_drop_query = drop_query; + /// FIXME We have to use global context to execute DROP query for inner table + /// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege + /// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant + /// looks like expected behaviour and we have tests for it. + auto drop_context = Context(global_context); + drop_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + if (auto txn = current_context.getZooKeeperMetadataTransaction()) + { + /// For Replicated database + drop_context.setQueryContext(const_cast(current_context)); + drop_context.initZooKeeperMetadataTransaction(txn, true); + } + InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context); + drop_interpreter.execute(); + } +} + } diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 4a67857767f..3c05c2788f1 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -26,6 +26,8 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, const Context &) const override; + static void executeDropQuery(ASTDropQuery::Kind kind, const Context & global_context, const Context & current_context, const StorageID & target_table_id, bool no_delay); + private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; ASTPtr query_ptr; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b859e654967..2ecf1e33560 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -345,6 +345,8 @@ public: */ virtual void drop() {} + virtual void dropInnerTableIfAny(bool /* no_delay */, const Context & /* context */) {} + /** Clear the table data and leave it empty. * Must be called under exclusive lock (lockExclusively). */ diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 556ede4ce7b..4b14035fad7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -4,10 +4,10 @@ #include #include #include - +#include +#include #include #include -#include #include @@ -124,7 +124,7 @@ void PostgreSQLReplicationHandler::startSynchronization() { nested_storages[table_name] = storage->getNested(); storage->setStorageMetadata(); - storage->setNestedLoaded(); + storage->setNestedStatus(true); } catch (...) { @@ -183,7 +183,7 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); copyData(input, *block_io.out); - storage_data.second->setNestedLoaded(); + storage_data.second->setNestedStatus(true); nested_storages[table_name] = nested_storage; /// This is needed if this method is called from reloadFromSnapshot() method below. @@ -401,7 +401,9 @@ std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapsh const auto & table_name = relation.second; auto * storage = storages[table_name]; sync_storages[table_name] = storage; - storage->dropNested(); + auto nested_storage = storage->getNested(); + storage->setNestedStatus(false); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, context, context, nested_storage->getStorageID(), true); } auto replication_connection = postgres::createReplicationConnection(connection_info); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index e4b87b8410b..b26ecf805f6 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace DB @@ -52,6 +53,8 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , replication_settings(std::move(replication_settings_)) , is_postgresql_replica_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") + , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) + , nested_context(makeNestedTableContext()) { setInMemoryMetadata(storage_metadata); @@ -70,16 +73,28 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, - StoragePtr nested_storage_, const Context & context_) : IStorage(table_id_) , global_context(context_) - , nested_storage(nested_storage_) , is_postgresql_replica_database(true) + , nested_table_id(table_id_) + , nested_context(makeNestedTableContext()) { } +StoragePtr StorageMaterializePostgreSQL::getNested() const +{ + return DatabaseCatalog::instance().getTable(nested_table_id, nested_context); +} + + +StoragePtr StorageMaterializePostgreSQL::tryGetNested() const +{ + return DatabaseCatalog::instance().tryGetTable(nested_table_id, nested_context); +} + + std::string StorageMaterializePostgreSQL::getNestedTableName() const { auto table_name = getStorageID().table_name; @@ -91,6 +106,17 @@ std::string StorageMaterializePostgreSQL::getNestedTableName() const } +void StorageMaterializePostgreSQL::setStorageMetadata() +{ + /// If it is a MaterializePostgreSQL database engine, then storage with engine MaterializePostgreSQL + /// gets its metadata when it is fetch from postges, but if inner tables exist (i.e. it is a server restart) + /// then metadata for storage needs to be set from inner table metadata. + auto nested_table = getNested(); + auto storage_metadata = nested_table->getInMemoryMetadataPtr(); + setInMemoryMetadata(*storage_metadata); +} + + std::shared_ptr StorageMaterializePostgreSQL::getMaterializedColumnsDeclaration( const String name, const String type, UInt64 default_value) { @@ -150,13 +176,6 @@ ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & da } -void StorageMaterializePostgreSQL::setStorageMetadata() -{ - auto storage_metadata = getNested()->getInMemoryMetadataPtr(); - setInMemoryMetadata(*storage_metadata); -} - - /// For single storage MaterializePostgreSQL get columns and primary key columns from storage definition. /// For database engine MaterializePostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same /// transaction with snapshot, which is used for initial tables dump. @@ -231,8 +250,8 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt columns_declare_list->set(columns_declare_list->columns, columns_expression_list); - columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", UInt64(1))); - columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", UInt64(1))); + columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", 1)); + columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", 1)); create_table_query->set(create_table_query->columns_list, columns_declare_list); @@ -255,14 +274,6 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) { - if (nested_loaded) - { - nested_storage = tryGetNested(); - - if (nested_storage) - return; - } - auto context = makeNestedTableContext(); const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); @@ -275,8 +286,6 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure { tryLogCurrentException(__PRETTY_FUNCTION__); } - - nested_storage = getNested(); } @@ -290,32 +299,6 @@ Context StorageMaterializePostgreSQL::makeNestedTableContext() const } -StoragePtr StorageMaterializePostgreSQL::getNested() -{ - if (nested_storage) - return nested_storage; - - auto context = makeNestedTableContext(); - nested_storage = DatabaseCatalog::instance().getTable( - StorageID(getStorageID().database_name, getNestedTableName()), context); - - return nested_storage; -} - - -StoragePtr StorageMaterializePostgreSQL::tryGetNested() -{ - if (nested_storage) - return nested_storage; - - auto context = makeNestedTableContext(); - nested_storage = DatabaseCatalog::instance().tryGetTable( - StorageID(getStorageID().database_name, getNestedTableName()), context); - - return nested_storage; -} - - void StorageMaterializePostgreSQL::startup() { if (!is_postgresql_replica_database) @@ -333,47 +316,23 @@ void StorageMaterializePostgreSQL::shutdown() } -void StorageMaterializePostgreSQL::shutdownFinal() +void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, const Context & context) { - if (is_postgresql_replica_database) - return; - if (replication_handler) replication_handler->shutdownFinal(); - if (nested_storage) - dropNested(); -} - - -void StorageMaterializePostgreSQL::dropNested() -{ - std::lock_guard lock(nested_mutex); - nested_loaded = false; - - auto table_id = nested_storage->getStorageID(); - auto ast_drop = std::make_shared(); - - ast_drop->kind = ASTDropQuery::Drop; - ast_drop->table = table_id.table_name; - ast_drop->database = table_id.database_name; - ast_drop->if_exists = true; - - auto context = makeNestedTableContext(); - auto interpreter = InterpreterDropQuery(ast_drop, context); - interpreter.execute(); - - nested_storage = nullptr; - LOG_TRACE(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Dropped (possibly temporarily) nested table {}", getNestedTableName()); + auto nested_table = getNested(); + if (nested_table && !is_postgresql_replica_database) + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, global_context, context, nested_table_id, no_delay); } NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const { - if (nested_storage) - return nested_storage->getVirtuals(); - - return {}; + return NamesAndTypesList{ + {"_sign", std::make_shared()}, + {"_version", std::make_shared()} + }; } @@ -386,26 +345,20 @@ Pipe StorageMaterializePostgreSQL::read( size_t max_block_size, unsigned num_streams) { - std::unique_lock lock(nested_mutex, std::defer_lock); + if (!nested_loaded) + return Pipe(); - if (nested_loaded && lock.try_lock()) - { - if (!nested_storage) - getNested(); + auto nested_table = getNested(); - return readFinalFromNestedStorage( - nested_storage, - column_names, - metadata_snapshot, - query_info, - context, - processed_stage, - max_block_size, - num_streams); - } - - LOG_WARNING(&Poco::Logger::get("StorageMaterializePostgreSQL"), "Nested table {} is unavailable or is not loaded yet", getNestedTableName()); - return Pipe(); + return readFinalFromNestedStorage( + nested_table, + column_names, + metadata_snapshot, + query_info, + context, + processed_stage, + max_block_size, + num_streams); } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 5bbea64133a..f311414c041 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -31,14 +31,16 @@ class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper replication_handler; std::atomic nested_loaded = false; - StoragePtr nested_storage; - std::mutex nested_mutex; - bool is_postgresql_replica_database = false; + StorageID nested_table_id; + const Context nested_context; }; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index c89187a46e2..75e34f97532 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -198,36 +197,6 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const } -static void executeDropQuery(ASTDropQuery::Kind kind, const Context & global_context, const Context & current_context, const StorageID & target_table_id, bool no_delay) -{ - if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) - { - /// We create and execute `drop` query for internal table. - auto drop_query = std::make_shared(); - drop_query->database = target_table_id.database_name; - drop_query->table = target_table_id.table_name; - drop_query->kind = kind; - drop_query->no_delay = no_delay; - drop_query->if_exists = true; - ASTPtr ast_drop_query = drop_query; - /// FIXME We have to use global context to execute DROP query for inner table - /// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege - /// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant - /// looks like expected behaviour and we have tests for it. - auto drop_context = Context(global_context); - drop_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - if (auto txn = current_context.getZooKeeperMetadataTransaction()) - { - /// For Replicated database - drop_context.setQueryContext(const_cast(current_context)); - drop_context.initZooKeeperMetadataTransaction(txn, true); - } - InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context); - drop_interpreter.execute(); - } -} - - void StorageMaterializedView::drop() { auto table_id = getStorageID(); @@ -235,19 +204,19 @@ void StorageMaterializedView::drop() if (!select_query.select_table_id.empty()) DatabaseCatalog::instance().removeDependency(select_query.select_table_id, table_id); - dropInnerTable(true, global_context); + dropInnerTableIfAny(true, global_context); } -void StorageMaterializedView::dropInnerTable(bool no_delay, const Context & context) +void StorageMaterializedView::dropInnerTableIfAny(bool no_delay, const Context & context) { if (has_inner_table && tryGetTargetTable()) - executeDropQuery(ASTDropQuery::Kind::Drop, global_context, context, target_table_id, no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, global_context, context, target_table_id, no_delay); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context & context, TableExclusiveLockHolder &) { if (has_inner_table) - executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, context, target_table_id, true); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, context, target_table_id, true); } void StorageMaterializedView::checkStatementCanBeForwarded() const diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index a5dc089d68e..cd89af154a9 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -37,7 +37,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void drop() override; - void dropInnerTable(bool no_delay, const Context & context); + void dropInnerTableIfAny(bool no_delay, const Context & context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; From 820a32d9396643f0e277bd2cfc37c10a33ec0d8f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 11 Apr 2021 06:00:47 +0000 Subject: [PATCH 064/105] Allow rename in case of single storage --- .../StorageMaterializePostgreSQL.cpp | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index b26ecf805f6..c5d45a335a2 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -362,6 +362,41 @@ Pipe StorageMaterializePostgreSQL::read( } +void StorageMaterializePostgreSQL::renameInMemory(const StorageID & new_table_id) +{ + auto old_table_id = getStorageID(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); + + if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database) + { + auto new_target_table_name = generateInnerTableName(new_table_id); + auto rename = std::make_shared(); + + ASTRenameQuery::Table from; + from.database = target_table_id.database_name; + from.table = target_table_id.table_name; + + ASTRenameQuery::Table to; + to.database = target_table_id.database_name; + to.table = new_target_table_name; + + ASTRenameQuery::Element elem; + elem.from = from; + elem.to = to; + rename->elements.emplace_back(elem); + + InterpreterRenameQuery(rename, global_context).execute(); + target_table_id.table_name = new_target_table_name; + } + + IStorage::renameInMemory(new_table_id); + const auto & select_query = metadata_snapshot->getSelectQuery(); + // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated + DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); +} + + void registerStorageMaterializePostgreSQL(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) From beae1c5fa472bb663adcd5bdc182ae268cd088cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 11 Apr 2021 07:44:40 +0000 Subject: [PATCH 065/105] Use ContextPtr --- src/Core/ya.make | 4 - .../DatabaseMaterializePostgreSQL.cpp | 36 +++--- .../DatabaseMaterializePostgreSQL.h | 13 +-- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterDropQuery.cpp | 12 +- src/Interpreters/InterpreterDropQuery.h | 2 +- src/Storages/IStorage.h | 2 +- .../MaterializePostgreSQLConsumer.cpp | 8 +- .../MaterializePostgreSQLConsumer.h | 4 +- .../PostgreSQLReplicationHandler.cpp | 12 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 +- .../StorageMaterializePostgreSQL.cpp | 103 +++++++++--------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 18 +-- .../ReadFinalForExternalReplicaStorage.cpp | 4 +- .../ReadFinalForExternalReplicaStorage.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 +- src/Storages/StorageMaterializedView.h | 2 +- 18 files changed, 114 insertions(+), 122 deletions(-) diff --git a/src/Core/ya.make b/src/Core/ya.make index 890ce20e7b3..004653d060e 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -31,10 +31,6 @@ SRCS( MySQL/PacketsProtocolText.cpp MySQL/PacketsReplication.cpp NamesAndTypes.cpp - PostgreSQL/PostgreSQLConnection.cpp - PostgreSQL/PostgreSQLConnectionPool.cpp - PostgreSQL/PostgreSQLPoolWithFailover.cpp - PostgreSQL/insertPostgreSQLValue.cpp PostgreSQLProtocol.cpp QueryProcessingStage.cpp Settings.cpp diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index ad60b6242ff..e5d61709387 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( - const Context & context_, + ContextPtr context_, const String & metadata_path_, UUID uuid_, const ASTStorage * database_engine_define_, @@ -58,7 +58,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() remote_database_name, connection->getConnectionInfo(), metadata_path + METADATA_SUFFIX, - global_context, + getContext(), settings->postgresql_replica_max_block_size.value, settings->postgresql_replica_allow_minimal_ddl, true, settings->postgresql_replica_tables_list.value); @@ -67,12 +67,12 @@ void DatabaseMaterializePostgreSQL::startSynchronization() for (const auto & table_name : tables_to_replicate) { - auto storage = tryGetTable(table_name, global_context); + auto storage = tryGetTable(table_name, getContext()); if (!storage) - storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), global_context); + storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), getContext()); - replication_handler->addStorage(table_name, storage->template as()); + replication_handler->addStorage(table_name, storage->as()); materialized_tables[table_name] = storage; } @@ -88,9 +88,9 @@ void DatabaseMaterializePostgreSQL::shutdown() } -void DatabaseMaterializePostgreSQL::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseMaterializePostgreSQL::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) { - DatabaseAtomic::loadStoredObjects(context, has_force_restore_data_flag, force_attach); + DatabaseAtomic::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); try { @@ -107,16 +107,16 @@ void DatabaseMaterializePostgreSQL::loadStoredObjects(Context & context, bool ha } -StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const Context & context) const +StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, ContextPtr local_context) const { /// When a nested ReplacingMergeTree table is managed from PostgreSQLReplicationHandler, its context is modified /// to show the type of managed table. - if (context.hasQueryContext()) + if (local_context->hasQueryContext()) { - auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; + auto storage_set = local_context->getQueryContext()->getQueryFactoriesInfo().storages; if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { - return DatabaseAtomic::tryGetTable(name, context); + return DatabaseAtomic::tryGetTable(name, local_context); } } @@ -132,14 +132,14 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, const } -void DatabaseMaterializePostgreSQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const String & name, const StoragePtr & table, const ASTPtr & query) { - if (context.hasQueryContext()) + if (local_context->hasQueryContext()) { - auto storage_set = context.getQueryContext().getQueryFactoriesInfo().storages; + auto storage_set = local_context->getQueryContext()->getQueryFactoriesInfo().storages; if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { - DatabaseAtomic::createTable(context, name, table, query); + DatabaseAtomic::createTable(local_context, name, table, query); return; } } @@ -156,7 +156,7 @@ void DatabaseMaterializePostgreSQL::stopReplication() } -void DatabaseMaterializePostgreSQL::drop(const Context & context) +void DatabaseMaterializePostgreSQL::drop(ContextPtr local_context) { if (replication_handler) replication_handler->shutdownFinal(); @@ -167,12 +167,12 @@ void DatabaseMaterializePostgreSQL::drop(const Context & context) if (metadata.exists()) metadata.remove(false); - DatabaseAtomic::drop(context); + DatabaseAtomic::drop(local_context); } DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( - const Context & /* context */, const DatabaseOnDisk::FilterByNameFunction & /* filter_by_table_name */) + ContextPtr /* context */, const DatabaseOnDisk::FilterByNameFunction & /* filter_by_table_name */) { Tables nested_tables; for (const auto & [table_name, storage] : materialized_tables) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index b87d281d7a5..17288be8fb2 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -20,7 +20,6 @@ namespace DB { -class Context; class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; @@ -30,7 +29,7 @@ class DatabaseMaterializePostgreSQL : public DatabaseAtomic public: DatabaseMaterializePostgreSQL( - const Context & context_, + ContextPtr context_, const String & metadata_path_, UUID uuid_, const ASTStorage * database_engine_define_, @@ -43,16 +42,16 @@ public: String getMetadataPath() const override { return metadata_path; } - void loadStoredObjects(Context &, bool, bool force_attach) override; + void loadStoredObjects(ContextPtr, bool, bool force_attach) override; DatabaseTablesIteratorPtr getTablesIterator( - const Context & context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override; + ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override; - StoragePtr tryGetTable(const String & name, const Context & context) const override; + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; - void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + void createTable(ContextPtr context, const String & name, const StoragePtr & table, const ASTPtr & query) override; - void drop(const Context & context) override; + void drop(ContextPtr local_context) override; void shutdown() override; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 6bee70fcbbb..1ddb8d53e79 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -149,7 +149,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr if (!table_checked && !checkPostgresTable(table_name)) return StoragePtr{}; - auto use_nulls = context.getSettingsRef().external_databases_use_nulls; + auto use_nulls = local_context->getSettingsRef().external_databases_use_nulls; auto connection = pool->get(); auto columns = fetchPostgreSQLTableStructure(connection->conn(), doubleQuoteString(table_name), use_nulls).columns; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d8c6cf7d1e4..1aeafcab04e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -216,13 +216,13 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "MaterializePostgreSQL" && !context.getSettingsRef().allow_experimental_database_materialize_postgresql && !internal) + if (create.storage->engine->name == "MaterializePostgreSQL" && !getContext()->getSettingsRef().allow_experimental_database_materialize_postgresql && !internal) { throw Exception("MaterializePostgreSQL is an experimental database engine. " "Enable allow_experimental_database_postgresql_replica to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext); + DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext()); if (create.uuid != UUIDHelpers::Nil) create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 41dc326b838..15e381d9191 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -436,7 +436,7 @@ void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const elem.query_kind = "Drop"; } -void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, const Context & global_context, const Context & current_context, const StorageID & target_table_id, bool no_delay) +void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay) { if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) { @@ -452,13 +452,13 @@ void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, const Conte /// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege /// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant /// looks like expected behaviour and we have tests for it. - auto drop_context = Context(global_context); - drop_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - if (auto txn = current_context.getZooKeeperMetadataTransaction()) + auto drop_context = Context::createCopy(global_context); + drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + if (auto txn = current_context->getZooKeeperMetadataTransaction()) { /// For Replicated database - drop_context.setQueryContext(const_cast(current_context)); - drop_context.initZooKeeperMetadataTransaction(txn, true); + drop_context->setQueryContext(current_context); + drop_context->initZooKeeperMetadataTransaction(txn, true); } InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context); drop_interpreter.execute(); diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 891c8378ff5..e50688fb7d4 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -26,7 +26,7 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; - static void executeDropQuery(ASTDropQuery::Kind kind, const Context & global_context, const Context & current_context, const StorageID & target_table_id, bool no_delay); + static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay); private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ea38eedf2b7..f5292c0b094 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -344,7 +344,7 @@ public: */ virtual void drop() {} - virtual void dropInnerTableIfAny(bool /* no_delay */, const Context & /* context */) {} + virtual void dropInnerTableIfAny(bool /* no_delay */, ContextPtr /* context */) {} /** Clear the table data and leave it empty. * Must be called under exclusive lock (lockExclusively). diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index ed317f03e60..0a597d942f7 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes } MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( - const Context & context_, + ContextPtr context_, postgres::ConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, @@ -491,9 +491,9 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptrtable_id = storage->getStorageID(); insert->columns = buffer.columnsAST; - auto insert_context(context); - insert_context.makeQueryContext(); - insert_context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + auto insert_context = Context::createCopy(context); + insert_context->makeQueryContext(); + insert_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); InterpreterInsertQuery interpreter(insert, insert_context, true); auto block_io = interpreter.execute(); diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index a52ecd73e07..7eeac16337e 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -27,7 +27,7 @@ public: using Storages = std::unordered_map; MaterializePostgreSQLConsumer( - const Context & context_, + ContextPtr context_, postgres::ConnectionPtr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, @@ -101,7 +101,7 @@ private: } Poco::Logger * log; - const Context & context; + ContextPtr context; const std::string replication_slot_name, publication_name; MaterializePostgreSQLMetadata metadata; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 4b14035fad7..4ea1dad2b14 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -17,13 +17,11 @@ namespace DB static const auto reschedule_ms = 500; -/// TODO: fetch replica identity index - PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & database_name_, const postgres::ConnectionInfo & connection_info_, const std::string & metadata_path_, - const Context & context_, + ContextPtr context_, const size_t max_block_size_, bool allow_minimal_ddl_, bool is_postgresql_replica_database_engine_, @@ -42,8 +40,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( replication_slot = fmt::format("{}_ch_replication_slot", database_name); publication_name = fmt::format("{}_ch_publication", database_name); - startup_task = context.getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); - consumer_task = context.getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); + startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); + consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); } @@ -169,7 +167,7 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na query_str = fmt::format("SELECT * FROM {}", storage_data.first); const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); - auto insert_context = storage_data.second->makeNestedTableContext(); + auto insert_context = storage_data.second->getNestedTableContext(); auto insert = std::make_shared(); insert->table_id = nested_storage->getStorageID(); @@ -384,7 +382,7 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( if (!is_postgresql_replica_database_engine) return nullptr; - auto use_nulls = context.getSettingsRef().external_databases_use_nulls; + auto use_nulls = context->getSettingsRef().external_databases_use_nulls; return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true, true)); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5557ae63f96..5a527179406 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -30,7 +30,7 @@ public: const std::string & database_name_, const postgres::ConnectionInfo & connection_info_, const std::string & metadata_path_, - const Context & context_, + ContextPtr context_, const size_t max_block_size_, bool allow_minimal_ddl_, bool is_postgresql_replica_database_engine_, @@ -79,7 +79,7 @@ private: PostgreSQLTableStructurePtr fetchTableStructure(std::shared_ptr tx, const std::string & table_name); Poco::Logger * log; - const Context & context; + ContextPtr context; /// Remote database name. const String database_name; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index c5d45a335a2..7a40b2003ed 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -45,11 +45,11 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const String & remote_table_name_, const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, - const Context & context_, + ContextPtr context_, std::unique_ptr replication_settings_) : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) , remote_table_name(remote_table_name_) - , global_context(context_.getGlobalContext()) , replication_settings(std::move(replication_settings_)) , is_postgresql_replica_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") @@ -65,7 +65,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( remote_database_name, connection_info, metadata_path, - global_context, + getContext(), replication_settings->postgresql_replica_max_block_size.value, replication_settings->postgresql_replica_allow_minimal_ddl.value, false); } @@ -73,9 +73,9 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, - const Context & context_) + ContextPtr context_) : IStorage(table_id_) - , global_context(context_) + , WithContext(context_->getGlobalContext()) , is_postgresql_replica_database(true) , nested_table_id(table_id_) , nested_context(makeNestedTableContext()) @@ -274,12 +274,11 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) { - auto context = makeNestedTableContext(); const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); try { - InterpreterCreateQuery interpreter(ast_create, context); + InterpreterCreateQuery interpreter(ast_create, nested_context); interpreter.execute(); } catch (...) @@ -289,11 +288,11 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure } -Context StorageMaterializePostgreSQL::makeNestedTableContext() const +std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext() const { - auto context(global_context); - context.makeQueryContext(); - context.addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + auto context = Context::createCopy(getContext()); + context->makeQueryContext(); + context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); return context; } @@ -316,14 +315,14 @@ void StorageMaterializePostgreSQL::shutdown() } -void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, const Context & context) +void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) { if (replication_handler) replication_handler->shutdownFinal(); auto nested_table = getNested(); if (nested_table && !is_postgresql_replica_database) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, global_context, context, nested_table_id, no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, nested_table_id, no_delay); } @@ -340,7 +339,7 @@ Pipe StorageMaterializePostgreSQL::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, - const Context & context, + ContextPtr context_, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) @@ -355,46 +354,46 @@ Pipe StorageMaterializePostgreSQL::read( column_names, metadata_snapshot, query_info, - context, + context_, processed_stage, max_block_size, num_streams); } -void StorageMaterializePostgreSQL::renameInMemory(const StorageID & new_table_id) -{ - auto old_table_id = getStorageID(); - auto metadata_snapshot = getInMemoryMetadataPtr(); - bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); - - if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database) - { - auto new_target_table_name = generateInnerTableName(new_table_id); - auto rename = std::make_shared(); - - ASTRenameQuery::Table from; - from.database = target_table_id.database_name; - from.table = target_table_id.table_name; - - ASTRenameQuery::Table to; - to.database = target_table_id.database_name; - to.table = new_target_table_name; - - ASTRenameQuery::Element elem; - elem.from = from; - elem.to = to; - rename->elements.emplace_back(elem); - - InterpreterRenameQuery(rename, global_context).execute(); - target_table_id.table_name = new_target_table_name; - } - - IStorage::renameInMemory(new_table_id); - const auto & select_query = metadata_snapshot->getSelectQuery(); - // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated - DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); -} +//void StorageMaterializePostgreSQL::renameInMemory(const StorageID & new_table_id) +//{ +// auto old_table_id = getStorageID(); +// auto metadata_snapshot = getInMemoryMetadataPtr(); +// bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); +// +// if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database) +// { +// auto new_target_table_name = generateInnerTableName(new_table_id); +// auto rename = std::make_shared(); +// +// ASTRenameQuery::Table from; +// from.database = target_table_id.database_name; +// from.table = target_table_id.table_name; +// +// ASTRenameQuery::Table to; +// to.database = target_table_id.database_name; +// to.table = new_target_table_name; +// +// ASTRenameQuery::Element elem; +// elem.from = from; +// elem.to = to; +// rename->elements.emplace_back(elem); +// +// InterpreterRenameQuery(rename, global_context).execute(); +// target_table_id.table_name = new_target_table_name; +// } +// +// IStorage::renameInMemory(new_table_id); +// const auto & select_query = metadata_snapshot->getSelectQuery(); +// // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated +// DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); +//} void registerStorageMaterializePostgreSQL(StorageFactory & factory) @@ -414,7 +413,7 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context); + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getContext()); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -427,9 +426,9 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) throw Exception("Storage MaterializePostgreSQL needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS); if (args.storage_def->primary_key) - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); else - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context); + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.getContext()); auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 5432); const String & remote_table = engine_args[2]->as().value.safeGet(); @@ -445,7 +444,7 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) return StorageMaterializePostgreSQL::create( args.table_id, remote_database, remote_table, connection_info, - metadata, args.context, + metadata, args.getContext(), std::move(postgresql_replication_settings)); }; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index f311414c041..3820ee0d66c 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -24,14 +23,14 @@ namespace DB { -class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage +class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage, WithContext { friend struct ext::shared_ptr_helper; public: StorageMaterializePostgreSQL( const StorageID & table_id_, - const Context & context_); + ContextPtr context_); String getName() const override { return "MaterializePostgreSQL"; } @@ -39,7 +38,7 @@ public: void shutdown() override; - void dropInnerTableIfAny(bool no_delay, const Context & context) override; + void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override; NamesAndTypesList getVirtuals() const override; @@ -47,7 +46,7 @@ public: const Names & column_names, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, - const Context & context, + ContextPtr context_, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; @@ -58,7 +57,9 @@ public: StoragePtr tryGetNested() const; - Context makeNestedTableContext() const; + ContextPtr getNestedTableContext() const { return nested_context; } + + std::shared_ptr makeNestedTableContext() const; void setNestedStatus(bool loaded) { nested_loaded.store(loaded); } @@ -73,7 +74,7 @@ protected: const String & remote_table_name, const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, - const Context & context_, + ContextPtr context_, std::unique_ptr replication_settings_); private: @@ -87,7 +88,6 @@ private: std::string getNestedTableName() const; std::string remote_table_name; - const Context global_context; std::unique_ptr replication_settings; std::unique_ptr replication_handler; @@ -95,7 +95,7 @@ private: std::atomic nested_loaded = false; bool is_postgresql_replica_database = false; StorageID nested_table_id; - const Context nested_context; + ContextPtr nested_context; }; } diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index 37b95eb5d6a..985b9104085 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -21,13 +21,13 @@ Pipe readFinalFromNestedStorage( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, SelectQueryInfo & query_info, - const Context & context, + ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned int num_streams) { NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - auto lock = nested_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto lock = nested_storage->lockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); Block nested_header = nested_metadata->getSampleBlock(); diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.h b/src/Storages/ReadFinalForExternalReplicaStorage.h index 2062392b22f..b54592159ef 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.h +++ b/src/Storages/ReadFinalForExternalReplicaStorage.h @@ -18,7 +18,7 @@ Pipe readFinalFromNestedStorage( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, SelectQueryInfo & query_info, - const Context & context, + ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned int num_streams); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9d1c172a5cd..666c8ca3749 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -208,10 +208,10 @@ void StorageMaterializedView::drop() dropInnerTableIfAny(true, getContext()); } -void StorageMaterializedView::dropInnerTableIfAny(bool no_delay, const Context & context) +void StorageMaterializedView::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) { if (has_inner_table && tryGetTargetTable()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), context, target_table_id, no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, no_delay); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 83140106b73..33aa6b9274d 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -37,7 +37,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; void drop() override; - void dropInnerTableIfAny(bool no_delay, ContextPtr context) override; + void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; From 027f67affb59293e4cabe13a0fdbceb47d3789b5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 11 Apr 2021 09:54:10 +0000 Subject: [PATCH 066/105] Fix checks --- .../fetchPostgreSQLTableStructure.cpp | 2 +- .../StorageMaterializePostgreSQL.cpp | 46 +++++++++---------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 8 ++-- 3 files changed, 27 insertions(+), 29 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 01ade1da180..7aaa7cc6f2a 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -201,7 +201,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "and a.attrelid = t.oid " "and a.attnum = ANY(ix.indkey) " "and t.relkind = 'r' " /// simple tables - "and t.relname = '{}' " /// Connection is alread done to a needed database, only table name is needed. + "and t.relname = '{}' " /// Connection is already done to a needed database, only table name is needed. "and ix.indisreplident = 't' " /// index is is replica identity index "ORDER BY a.attname", /// column names postgres_table_name); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 7a40b2003ed..c97e3529deb 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -23,7 +23,6 @@ #include #include #include -#include namespace DB @@ -51,10 +50,10 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , WithContext(context_->getGlobalContext()) , remote_table_name(remote_table_name_) , replication_settings(std::move(replication_settings_)) - , is_postgresql_replica_database( + , is_materialize_postgresql_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) - , nested_context(makeNestedTableContext()) + , nested_context(makeNestedTableContext(context_->getGlobalContext())) { setInMemoryMetadata(storage_metadata); @@ -76,9 +75,9 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , is_postgresql_replica_database(true) + , is_materialize_postgresql_database(true) , nested_table_id(table_id_) - , nested_context(makeNestedTableContext()) + , nested_context(makeNestedTableContext(context_->getGlobalContext())) { } @@ -99,7 +98,7 @@ std::string StorageMaterializePostgreSQL::getNestedTableName() const { auto table_name = getStorageID().table_name; - if (!is_postgresql_replica_database) + if (!is_materialize_postgresql_database) table_name += NESTED_STORAGE_SUFFIX; return table_name; @@ -195,7 +194,7 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt const auto & columns = metadata_snapshot->getColumns(); NamesAndTypesList ordinary_columns_and_types; - if (!is_postgresql_replica_database) + if (!is_materialize_postgresql_database) { ordinary_columns_and_types = columns.getOrdinary(); } @@ -288,19 +287,19 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure } -std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext() const +std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) const { - auto context = Context::createCopy(getContext()); - context->makeQueryContext(); - context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + auto new_context = Context::createCopy(from_context); + new_context->makeQueryContext(); + new_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - return context; + return new_context; } void StorageMaterializePostgreSQL::startup() { - if (!is_postgresql_replica_database) + if (!is_materialize_postgresql_database) { replication_handler->addStorage(remote_table_name, this); replication_handler->startup(); @@ -321,7 +320,7 @@ void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr replication_handler->shutdownFinal(); auto nested_table = getNested(); - if (nested_table && !is_postgresql_replica_database) + if (nested_table && !is_materialize_postgresql_database) InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, nested_table_id, no_delay); } @@ -365,31 +364,32 @@ Pipe StorageMaterializePostgreSQL::read( //{ // auto old_table_id = getStorageID(); // auto metadata_snapshot = getInMemoryMetadataPtr(); -// bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); // -// if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database) +// IStorage::renameInMemory(new_table_id); +// auto nested_table = tryGetNested(); +// +// if (nested_table) // { -// auto new_target_table_name = generateInnerTableName(new_table_id); +// auto new_nested_table_name = getNestedTableName(); // auto rename = std::make_shared(); // // ASTRenameQuery::Table from; -// from.database = target_table_id.database_name; -// from.table = target_table_id.table_name; +// from.database = nested_table_id.database_name; +// from.table = nested_table_id.table_name; // // ASTRenameQuery::Table to; -// to.database = target_table_id.database_name; -// to.table = new_target_table_name; +// to.database = nested_table_id.database_name; +// to.table = new_nested_table_name; // // ASTRenameQuery::Element elem; // elem.from = from; // elem.to = to; // rename->elements.emplace_back(elem); // -// InterpreterRenameQuery(rename, global_context).execute(); +// InterpreterRenameQuery(rename, getContext()).execute(); // target_table_id.table_name = new_target_table_name; // } // -// IStorage::renameInMemory(new_table_id); // const auto & select_query = metadata_snapshot->getSelectQuery(); // // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated // DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 3820ee0d66c..686cee7a1e6 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -59,8 +59,6 @@ public: ContextPtr getNestedTableContext() const { return nested_context; } - std::shared_ptr makeNestedTableContext() const; - void setNestedStatus(bool loaded) { nested_loaded.store(loaded); } bool isNestedLoaded() { return nested_loaded.load(); } @@ -87,13 +85,13 @@ private: std::string getNestedTableName() const; - std::string remote_table_name; + std::shared_ptr makeNestedTableContext(ContextPtr from_context) const; + std::string remote_table_name; std::unique_ptr replication_settings; std::unique_ptr replication_handler; - std::atomic nested_loaded = false; - bool is_postgresql_replica_database = false; + bool is_materialize_postgresql_database = false; StorageID nested_table_id; ContextPtr nested_context; }; From c968ccb391f99ddfa55cf41532457301594b9a4e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 11 Apr 2021 17:19:20 +0000 Subject: [PATCH 067/105] Better dependent name for nested in case of single storage --- .../StorageMaterializePostgreSQL.cpp | 51 ++++--------------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 4 +- .../test_storage_postgresql_replica/test.py | 32 ++++++++++++ 3 files changed, 44 insertions(+), 43 deletions(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index c97e3529deb..72611428326 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree"; +static const auto NESTED_TABLE_SUFFIX = "_nested"; StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( @@ -55,6 +55,9 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) , nested_context(makeNestedTableContext(context_->getGlobalContext())) { + if (table_id_.uuid == UUIDHelpers::Nil) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializePostgreSQL is allowed only for Atomic database"); + setInMemoryMetadata(storage_metadata); auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath() @@ -96,12 +99,12 @@ StoragePtr StorageMaterializePostgreSQL::tryGetNested() const std::string StorageMaterializePostgreSQL::getNestedTableName() const { - auto table_name = getStorageID().table_name; + auto table_id = getStorageID(); - if (!is_materialize_postgresql_database) - table_name += NESTED_STORAGE_SUFFIX; + if (is_materialize_postgresql_database) + return table_id.table_name; - return table_name; + return toString(table_id.uuid) + NESTED_TABLE_SUFFIX; } @@ -287,7 +290,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure } -std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) const +std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) { auto new_context = Context::createCopy(from_context); new_context->makeQueryContext(); @@ -360,42 +363,6 @@ Pipe StorageMaterializePostgreSQL::read( } -//void StorageMaterializePostgreSQL::renameInMemory(const StorageID & new_table_id) -//{ -// auto old_table_id = getStorageID(); -// auto metadata_snapshot = getInMemoryMetadataPtr(); -// -// IStorage::renameInMemory(new_table_id); -// auto nested_table = tryGetNested(); -// -// if (nested_table) -// { -// auto new_nested_table_name = getNestedTableName(); -// auto rename = std::make_shared(); -// -// ASTRenameQuery::Table from; -// from.database = nested_table_id.database_name; -// from.table = nested_table_id.table_name; -// -// ASTRenameQuery::Table to; -// to.database = nested_table_id.database_name; -// to.table = new_nested_table_name; -// -// ASTRenameQuery::Element elem; -// elem.from = from; -// elem.to = to; -// rename->elements.emplace_back(elem); -// -// InterpreterRenameQuery(rename, getContext()).execute(); -// target_table_id.table_name = new_target_table_name; -// } -// -// const auto & select_query = metadata_snapshot->getSelectQuery(); -// // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated -// DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); -//} - - void registerStorageMaterializePostgreSQL(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 686cee7a1e6..079061a69d4 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -65,6 +65,8 @@ public: void setStorageMetadata(); + void renameNested(); + protected: StorageMaterializePostgreSQL( const StorageID & table_id_, @@ -85,7 +87,7 @@ private: std::string getNestedTableName() const; - std::shared_ptr makeNestedTableContext(ContextPtr from_context) const; + static std::shared_ptr makeNestedTableContext(ContextPtr from_context); std::string remote_table_name; std::unique_ptr replication_settings; diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index bca4f159cf6..678c7384c1d 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -430,6 +430,38 @@ def test_clickhouse_restart(started_cluster): assert(int(result) == 100050) +def test_rename_table(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + ENGINE = MaterializePostgreSQL( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; ''') + + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while int(result) != 50: + time.sleep(0.5) + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + instance.query('RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed') + + result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') + while int(result) != 50: + time.sleep(0.5) + result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50, 50)") + + result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') + while int(result) != 100: + time.sleep(0.5) + result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 01075677cfa53dccf5af33e4b3249a6917ab611e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 11 Apr 2021 19:58:33 +0000 Subject: [PATCH 068/105] Better --- .../materialize-postgresql.md | 5 +- src/Core/PostgreSQL/PostgreSQLConnection.h | 2 +- src/DataStreams/PostgreSQLBlockInputStream.h | 13 +++-- .../DatabaseMaterializePostgreSQL.cpp | 17 ++++-- .../DatabaseMaterializePostgreSQL.h | 2 + .../fetchPostgreSQLTableStructure.cpp | 54 +++++++++++-------- .../fetchPostgreSQLTableStructure.h | 9 ++-- .../PostgreSQLReplicationHandler.cpp | 41 ++++++++------ .../PostgreSQL/PostgreSQLReplicationHandler.h | 14 ++--- src/Storages/StoragePostgreSQL.h | 1 - .../test.py | 50 ----------------- 11 files changed, 92 insertions(+), 116 deletions(-) diff --git a/docs/en/engines/database-engines/materialize-postgresql.md b/docs/en/engines/database-engines/materialize-postgresql.md index b3516001929..79dccabc287 100644 --- a/docs/en/engines/database-engines/materialize-postgresql.md +++ b/docs/en/engines/database-engines/materialize-postgresql.md @@ -7,9 +7,9 @@ toc_title: MaterializePostgreSQL ## Creating a Database {#creating-a-database} -## Requirements +## Requirements {#requirements} -Each replicated table must have one of the following **replica identity**: +- Each replicated table must have one of the following **replica identity**: 1. **default** (primary key) @@ -38,3 +38,4 @@ WHERE oid = 'postgres_table'::regclass; ``` +- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. diff --git a/src/Core/PostgreSQL/PostgreSQLConnection.h b/src/Core/PostgreSQL/PostgreSQLConnection.h index dfed426b462..f884e93669d 100644 --- a/src/Core/PostgreSQL/PostgreSQLConnection.h +++ b/src/Core/PostgreSQL/PostgreSQLConnection.h @@ -25,7 +25,7 @@ class Connection; using ConnectionPtr = std::shared_ptr; -/// Connection string and address without login/password (for error logs) +/// Connection string and address without credentials (for logs) using ConnectionInfo = std::pair; ConnectionInfo formatConnectionString( diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 5c637015f18..f320e2caeb5 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -28,6 +27,12 @@ public: const Block & sample_block, const UInt64 max_block_size_); + String getName() const override { return "PostgreSQL"; } + Block getHeader() const override { return description.sample_block.cloneEmpty(); } + + void readPrefix() override; + +protected: PostgreSQLBlockInputStream( std::shared_ptr tx_, const std::string & query_str_, @@ -35,12 +40,6 @@ public: const UInt64 max_block_size_, bool auto_commit_); - String getName() const override { return "PostgreSQL"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - - void readPrefix() override; - -protected: String query_str; std::shared_ptr tx; std::unique_ptr stream; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index e5d61709387..c8e93616ead 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -60,7 +60,8 @@ void DatabaseMaterializePostgreSQL::startSynchronization() metadata_path + METADATA_SUFFIX, getContext(), settings->postgresql_replica_max_block_size.value, - settings->postgresql_replica_allow_minimal_ddl, true, + settings->postgresql_replica_allow_minimal_ddl, + /* is_materialize_postgresql_database = */ true, settings->postgresql_replica_tables_list.value); std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->getRef()); @@ -123,9 +124,9 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte /// Note: In select query we call MaterializePostgreSQL table and it calls tryGetTable from its nested. std::lock_guard lock(tables_mutex); auto table = materialized_tables.find(name); - /// Here it is possible that nested table is temporarily out of reach, but return storage anyway, - /// it will not allow to read if nested is unavailable at the moment - if (table != materialized_tables.end()) + + /// Nested table is not created immediately. Consider that table exists only if nested table exists. + if (table != materialized_tables.end() && table->second->as()->isNestedLoaded()) return table->second; return StoragePtr{}; @@ -177,7 +178,7 @@ DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( Tables nested_tables; for (const auto & [table_name, storage] : materialized_tables) { - auto nested_storage = storage->template as()->tryGetNested(); + auto nested_storage = storage->as()->tryGetNested(); if (nested_storage) nested_tables[table_name] = nested_storage; @@ -186,6 +187,12 @@ DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( return std::make_unique(nested_tables, database_name); } + +void DatabaseMaterializePostgreSQL::renameTable(ContextPtr /* context_ */, const String & /* name */, IDatabase & /* to_database */, const String & /* to_name */, bool /* exchange */, bool /* dictionary */) +{ + throw Exception("MaterializePostgreSQL database does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); +} + } #endif diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index 17288be8fb2..bdfe54ace13 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -51,6 +51,8 @@ public: void createTable(ContextPtr context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; + void drop(ContextPtr local_context) override; void shutdown() override; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 7aaa7cc6f2a..1f8b08d3807 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -26,14 +26,14 @@ namespace ErrorCodes } -std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection) +template +std::unordered_set fetchPostgreSQLTablesList(T & tx) { std::unordered_set tables; std::string query = "SELECT tablename FROM pg_catalog.pg_tables " "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'"; - pqxx::read_transaction tx(connection); - for (auto table_name : tx.stream(query)) + for (auto table_name : tx.template stream(query)) tables.insert(std::get<0>(table_name)); return tables; @@ -112,13 +112,13 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl template std::shared_ptr readNamesAndTypesList( - std::shared_ptr tx, const String & postgres_table_name, const String & query, bool use_nulls, bool only_names_and_types) + T & tx, const String & postgres_table_name, const String & query, bool use_nulls, bool only_names_and_types) { auto columns = NamesAndTypesList(); try { - pqxx::stream_from stream(*tx, pqxx::from_query, std::string_view(query)); + pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); if (only_names_and_types) { @@ -158,7 +158,7 @@ std::shared_ptr readNamesAndTypesList( template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key, bool with_replica_identity_index) + T & tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key, bool with_replica_identity_index) { PostgreSQLTableStructure table; @@ -213,29 +213,37 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( } -template -PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, - bool with_primary_key, bool with_replica_identity_index); - - -template -PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, - bool with_primary_key, bool with_replica_identity_index); - - PostgreSQLTableStructure fetchPostgreSQLTableStructure( pqxx::connection & connection, const String & postgres_table_name, bool use_nulls) { - auto tx = std::make_shared(connection); - auto table = fetchPostgreSQLTableStructure(tx, postgres_table_name, use_nulls, false, false); - tx->commit(); - - return table; + postgres::Transaction tx(connection); + return fetchPostgreSQLTableStructure(tx.getRef(), postgres_table_name, use_nulls, false, false); } +std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection) +{ + postgres::Transaction tx(connection); + return fetchPostgreSQLTablesList(tx.getRef()); +} + + +template +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + pqxx::ReadTransaction & tx, const String & postgres_table_name, bool use_nulls, + bool with_primary_key, bool with_replica_identity_index); + +template +PostgreSQLTableStructure fetchPostgreSQLTableStructure( + pqxx::ReplicationTransaction & tx, const String & postgres_table_name, bool use_nulls, + bool with_primary_key, bool with_replica_identity_index); + +template +std::unordered_set fetchPostgreSQLTablesList(pqxx::work & tx); + +template +std::unordered_set fetchPostgreSQLTablesList(pqxx::ReadTransaction & tx); + } #endif diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index bbcb9cd192f..2853e0a8ea4 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -12,8 +12,6 @@ namespace DB { -std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection); - struct PostgreSQLTableStructure { std::shared_ptr columns; @@ -23,14 +21,19 @@ struct PostgreSQLTableStructure using PostgreSQLTableStructurePtr = std::unique_ptr; +std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection); + PostgreSQLTableStructure fetchPostgreSQLTableStructure( pqxx::connection & connection, const String & postgres_table_name, bool use_nulls); template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - std::shared_ptr tx, const String & postgres_table_name, bool use_nulls, + T & tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false, bool with_replica_identity_index = false); +template +std::unordered_set fetchPostgreSQLTablesList(T & tx); + } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 4ea1dad2b14..c1056f925bb 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -87,7 +87,10 @@ void PostgreSQLReplicationHandler::shutdown() void PostgreSQLReplicationHandler::startSynchronization() { - createPublicationIfNeeded(connection->getRef()); + { + postgres::Transaction tx(connection->getRef()); + createPublicationIfNeeded(tx.getRef()); + } auto replication_connection = postgres::createReplicationConnection(connection_info); postgres::Transaction tx(replication_connection->getRef()); @@ -159,7 +162,7 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); - storage_data.second->createNestedIfNeeded(fetchTableStructure(tx, table_name)); + storage_data.second->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); auto nested_storage = storage_data.second->getNested(); /// Load from snapshot, which will show table state before creation of replication slot. @@ -233,14 +236,12 @@ bool PostgreSQLReplicationHandler::isPublicationExist(pqxx::work & tx) } -void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::connection & connection_) +void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bool create_without_check) { if (new_publication_created) return; - postgres::Transaction tx(connection_); - - if (!isPublicationExist(tx.getRef())) + if (create_without_check || !isPublicationExist(tx)) { if (tables_list.empty()) { @@ -349,27 +350,33 @@ void PostgreSQLReplicationHandler::shutdownFinal() } +/// Used by MaterializePostgreSQL database engine. NameSet PostgreSQLReplicationHandler::fetchRequiredTables(pqxx::connection & connection_) { - if (tables_list.empty()) + postgres::Transaction tx(connection_); + bool publication_exists = isPublicationExist(tx.getRef()); + + if (tables_list.empty() && !publication_exists) { - return fetchPostgreSQLTablesList(connection_); - } - else - { - createPublicationIfNeeded(connection_); - return fetchTablesFromPublication(connection_); + /// Fetch all tables list from database. Publication does not exist yet, which means + /// that no replication took place. Publication will be created in + /// startSynchronization method. + return fetchPostgreSQLTablesList(tx.getRef()); } + + if (!publication_exists) + createPublicationIfNeeded(tx.getRef(), /* create_without_check = */ true); + + return fetchTablesFromPublication(tx.getRef()); } -NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::connection & connection_) +NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx) { std::string query = fmt::format("SELECT tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); std::unordered_set tables; - postgres::Transaction tx(connection_); - for (auto table_name : tx.getRef().stream(query)) + for (auto table_name : tx.stream(query)) tables.insert(std::get<0>(table_name)); return tables; @@ -377,7 +384,7 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::connectio PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( - std::shared_ptr tx, const std::string & table_name) + pqxx::ReplicationTransaction & tx, const std::string & table_name) { if (!is_postgresql_replica_database_engine) return nullptr; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5a527179406..43f0067aed7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -52,18 +52,20 @@ public: private: using Storages = std::unordered_map; - void createPublicationIfNeeded(pqxx::connection & connection_); - bool isPublicationExist(pqxx::work & tx); + void createPublicationIfNeeded(pqxx::work & tx, bool create_without_check = false); + + NameSet fetchTablesFromPublication(pqxx::work & tx); + + void dropPublication(pqxx::nontransaction & ntx); + bool isReplicationSlotExist(pqxx::nontransaction & tx, std::string & slot_name); void createReplicationSlot(pqxx::nontransaction & tx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); void dropReplicationSlot(pqxx::nontransaction & tx, bool temporary = false); - void dropPublication(pqxx::nontransaction & ntx); - void waitConnectionAndStart(); void startSynchronization(); @@ -72,11 +74,9 @@ private: NameSet loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages); - NameSet fetchTablesFromPublication(pqxx::connection & connection_); - std::unordered_map reloadFromSnapshot(const std::vector> & relation_data); - PostgreSQLTableStructurePtr fetchTableStructure(std::shared_ptr tx, const std::string & table_name); + PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const std::string & table_name); Poco::Logger * log; ContextPtr context; diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 9bf5a001f1b..fc57aded197 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -10,7 +10,6 @@ #include #include #include -#include namespace DB diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index cf93a3e1b1c..503e12c890f 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -303,56 +303,6 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') -@pytest.mark.timeout(320) -def test_table_schema_changes(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(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); - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) - - instance.query( - """CREATE DATABASE test_database - ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS postgresql_replica_allow_minimal_ddl = 1; - """) - - 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)) - - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - - expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); - - altered_table = random.randint(0, 4) - cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) - - for i in range(NUM_TABLES): - cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) - cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) - - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - - for i in range(NUM_TABLES): - if i != altered_table: - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) - else: - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) - - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - - for i in range(NUM_TABLES): - cursor.execute('drop table postgresql_replica_{};'.format(i)) - - instance.query("DROP DATABASE test_database") - - @pytest.mark.timeout(120) def test_changing_replica_identity_value(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") From 6413d7bac6b27a8ee268e3993268c99513609331 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 11 Apr 2021 20:26:59 +0000 Subject: [PATCH 069/105] Doc --- .../database-engines/materialize-postgresql.md | 2 +- .../integrations/materialize-postgresql.md | 16 ++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) create mode 100644 docs/en/engines/table-engines/integrations/materialize-postgresql.md diff --git a/docs/en/engines/database-engines/materialize-postgresql.md b/docs/en/engines/database-engines/materialize-postgresql.md index 79dccabc287..5f1ee614704 100644 --- a/docs/en/engines/database-engines/materialize-postgresql.md +++ b/docs/en/engines/database-engines/materialize-postgresql.md @@ -35,7 +35,7 @@ postgres# SELECT CASE relreplident END AS replica_identity FROM pg_class WHERE oid = 'postgres_table'::regclass; - ``` - Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. + diff --git a/docs/en/engines/table-engines/integrations/materialize-postgresql.md b/docs/en/engines/table-engines/integrations/materialize-postgresql.md new file mode 100644 index 00000000000..e3cbfbb087b --- /dev/null +++ b/docs/en/engines/table-engines/integrations/materialize-postgresql.md @@ -0,0 +1,16 @@ +--- +toc_priority: 12 +toc_title: MateriaziePostgreSQL +--- + +# MaterializePostgreSQL {#materialize-postgresql} + +## Creating a Table {#creating-a-table} + +## Requirements {#requirements} + +- A table with engine `MaterializePostgreSQL` must have a primary key - the same as a replica identity index of a postgres table (See [details on replica identity index](../../database-engines/materialize-postgresql.md#requirements)). + +- Only database `Atomic` is allowed. + +- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. From 14be3a07839891512918466cb3ee938cb1a1a7d7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 11:49:45 +0000 Subject: [PATCH 070/105] Better reload from snapshot --- .../DatabaseMaterializePostgreSQL.cpp | 1 + .../MaterializePostgreSQLConsumer.cpp | 8 +- .../MaterializePostgreSQLConsumer.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 177 ++++++++-------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 20 +- .../StorageMaterializePostgreSQL.cpp | 190 ++++++++++-------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 3 + 7 files changed, 216 insertions(+), 185 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index c8e93616ead..7d493d3dcf3 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -56,6 +56,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() { replication_handler = std::make_unique( remote_database_name, + database_name, connection->getConnectionInfo(), metadata_path + METADATA_SUFFIX, getContext(), diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 0a597d942f7..bd4e5c0cbe5 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -675,13 +675,9 @@ void MaterializePostgreSQLConsumer::updateNested(const String & table_name, Stor } -void MaterializePostgreSQLConsumer::updateSkipList(const std::unordered_map & tables_with_lsn) +void MaterializePostgreSQLConsumer::updateSkipList(Int32 table_id, const String & table_start_lsn) { - for (const auto & [relation_id, lsn] : tables_with_lsn) - { - if (!lsn.empty()) - skip_list[relation_id] = lsn; /// start_lsn - } + skip_list[table_id] = table_start_lsn; } } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 7eeac16337e..3bef0c717ba 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -43,7 +43,7 @@ public: void updateNested(const String & table_name, StoragePtr nested_storage); - void updateSkipList(const std::unordered_map & tables_with_lsn); + void updateSkipList(Int32 table_id, const String & table_start_lsn); private: bool readFromReplicationSlot(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index c1056f925bb..6607bb6d3f7 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -15,10 +15,12 @@ namespace DB { static const auto reschedule_ms = 500; +static const auto TMP_SUFFIX = "_tmp"; PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( - const std::string & database_name_, + const String & remote_database_name_, + const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, const std::string & metadata_path_, ContextPtr context_, @@ -28,7 +30,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const String tables_list_) : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , context(context_) - , database_name(database_name_) + , remote_database_name(remote_database_name_) + , current_database_name(current_database_name_) , metadata_path(metadata_path_) , connection_info(connection_info_) , max_block_size(max_block_size_) @@ -37,8 +40,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , tables_list(tables_list_) , connection(std::make_shared(connection_info_)) { - replication_slot = fmt::format("{}_ch_replication_slot", database_name); - publication_name = fmt::format("{}_ch_publication", database_name); + replication_slot = fmt::format("{}_ch_replication_slot", current_database_name); + publication_name = fmt::format("{}_ch_publication", current_database_name); startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); @@ -47,7 +50,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage) { - storages[table_name] = storage; + materialized_storages[table_name] = storage; } @@ -95,40 +98,59 @@ void PostgreSQLReplicationHandler::startSynchronization() auto replication_connection = postgres::createReplicationConnection(connection_info); postgres::Transaction tx(replication_connection->getRef()); + /// List of nested tables (table_name -> nested_storage), which is passed to replication consumer. + std::unordered_map nested_storages; std::string snapshot_name, start_lsn; auto initial_sync = [&]() { createReplicationSlot(tx.getRef(), start_lsn, snapshot_name); - loadFromSnapshot(snapshot_name, storages); + + for (const auto & [table_name, storage] : materialized_storages) + { + try + { + nested_storages[table_name] = loadFromSnapshot(snapshot_name, table_name, storage->as ()); + } + catch (Exception & e) + { + e.addMessage("while loading table {}.{}", remote_database_name, table_name); + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } }; - /// Replication slot should be deleted with drop table only and created only once, reused after detach. + /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, + /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. + /// Recreation of a replication slot imposes reloading of all tables. if (!isReplicationSlotExist(tx.getRef(), replication_slot)) { initial_sync(); } else if (!Poco::File(metadata_path).exists() || new_publication_created) { - /// In case of some failure, the following cases are possible (since publication and replication slot are reused): - /// 1. If replication slot exists and metadata file (where last synced version is written) does not exist, it is not ok. - /// 2. If created a new publication and replication slot existed before it was created, it is not ok. + /// There are the following cases, which mean that something non-intentioanal happened. + /// 1. If replication slot exists and metadata file does not exist, it is not ok. + /// 2. If replication slot exists before publication is created. dropReplicationSlot(tx.getRef()); initial_sync(); } else { - LOG_TRACE(log, "Restoring {} tables...", storages.size()); - for (const auto & [table_name, storage] : storages) + /// Synchronization and initial load already took place.c + LOG_TRACE(log, "Loading {} tables...", materialized_storages.size()); + for (const auto & [table_name, storage] : materialized_storages) { + auto materialized_storage = storage->as (); try { - nested_storages[table_name] = storage->getNested(); - storage->setStorageMetadata(); - storage->setNestedStatus(true); + nested_storages[table_name] = materialized_storage->getNested(); + materialized_storage->setStorageMetadata(); + materialized_storage->setNestedStatus(true); } - catch (...) + catch (Exception & e) { + e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -146,60 +168,47 @@ void PostgreSQLReplicationHandler::startSynchronization() nested_storages); consumer_task->activateAndSchedule(); + + /// Do not rely anymore on saved storage pointers. + materialized_storages.clear(); } -NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages) +StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializePostgreSQL * materialized_storage) { - NameSet success_tables; - for (const auto & storage_data : sync_storages) - { - try - { - auto tx = std::make_shared(connection->getRef()); - const auto & table_name = storage_data.first; + auto tx = std::make_shared(connection->getRef()); - std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); - tx->exec(query_str); + std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); + tx->exec(query_str); - storage_data.second->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); - auto nested_storage = storage_data.second->getNested(); + /// Load from snapshot, which will show table state before creation of replication slot. + /// Already connected to needed database, no need to add it to query. + query_str = fmt::format("SELECT * FROM {}", table_name); - /// Load from snapshot, which will show table state before creation of replication slot. - /// Already connected to needed database, no need to add it to query. - query_str = fmt::format("SELECT * FROM {}", storage_data.first); + /// If table schema has changed, the table stops consuming changed from replication stream. + /// If `allow_minimal_ddl` is true, create a new table in the background, load new table schema + /// and all data from scratch. Then execute REPLACE query with Nested table. + /// This is only allowed for MaterializePostgreSQL database engine. + materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); + auto nested_storage = materialized_storage->getNested(); + auto insert_context = materialized_storage->getNestedTableContext(); - const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); - auto insert_context = storage_data.second->getNestedTableContext(); + auto insert = std::make_shared(); + insert->table_id = nested_storage->getStorageID(); - auto insert = std::make_shared(); - insert->table_id = nested_storage->getStorageID(); + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); - InterpreterInsertQuery interpreter(insert, insert_context); - auto block_io = interpreter.execute(); + const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); + auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - PostgreSQLTransactionBlockInputStream input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); + PostgreSQLTransactionBlockInputStream input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); + assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); + copyData(input, *block_io.out); - assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); - copyData(input, *block_io.out); + materialized_storage->setNestedStatus(true); - storage_data.second->setNestedStatus(true); - nested_storages[table_name] = nested_storage; - - /// This is needed if this method is called from reloadFromSnapshot() method below. - success_tables.insert(table_name); - if (consumer) - consumer->updateNested(table_name, nested_storage); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - LOG_DEBUG(log, "Table dump end"); - return success_tables; + return nested_storage; } @@ -210,7 +219,7 @@ void PostgreSQLReplicationHandler::consumerFunc() bool schedule_now = consumer->consume(skipped_tables); if (!skipped_tables.empty()) - consumer->updateSkipList(reloadFromSnapshot(skipped_tables)); + reloadFromSnapshot(skipped_tables); if (stop_synchronization) return; @@ -245,7 +254,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bo { if (tables_list.empty()) { - for (const auto & storage_data : storages) + for (const auto & storage_data : materialized_storages) { if (!tables_list.empty()) tables_list += ", "; @@ -394,43 +403,53 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( } -std::unordered_map PostgreSQLReplicationHandler::reloadFromSnapshot( - const std::vector> & relation_data) +void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector> & relation_data) { - std::unordered_map tables_start_lsn; try { - Storages sync_storages; - for (const auto & relation : relation_data) - { - const auto & table_name = relation.second; - auto * storage = storages[table_name]; - sync_storages[table_name] = storage; - auto nested_storage = storage->getNested(); - storage->setNestedStatus(false); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, context, context, nested_storage->getStorageID(), true); - } - auto replication_connection = postgres::createReplicationConnection(connection_info); postgres::Transaction tx(replication_connection->getRef()); std::string snapshot_name, start_lsn; createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, true); - /// This snapshot is valid up to the end of the transaction, which exported it. - auto success_tables = loadFromSnapshot(snapshot_name, sync_storages); - for (const auto & relation : relation_data) + for (const auto & [table_id, table_name] : relation_data) { - if (success_tables.find(relation.second) != success_tables.end()) - tables_start_lsn[relation.first] = start_lsn; + auto materialized_storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context); + StoragePtr temp_materialized_storage = materialized_storage->as ()->createTemporary(); + + /// This snapshot is valid up to the end of the transaction, which exported it. + StoragePtr nested_storage = loadFromSnapshot(snapshot_name, table_name, + temp_materialized_storage->as ()); + consumer->updateNested(table_name, nested_storage); + consumer->updateSkipList(table_id, start_lsn); + replaceMaterializedTable(table_name); } } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } +} - return tables_start_lsn; + +void PostgreSQLReplicationHandler::replaceMaterializedTable(const String & table_name) +{ + auto ast_replace = std::make_shared(); + + auto outdated_storage = materialized_storages[table_name]; + auto table_id = outdated_storage->getStorageID(); + + ast_replace->replace_table = true; + + ast_replace->table = table_id.table_name; + ast_replace->database = table_id.database_name; + + ast_replace->as_table = table_id.table_name + TMP_SUFFIX; + ast_replace->as_database = table_id.database_name; + + InterpreterCreateQuery interpreter(ast_replace, context); + interpreter.execute(); } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 43f0067aed7..76aed35ba80 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -27,9 +27,10 @@ class PostgreSQLReplicationHandler { public: PostgreSQLReplicationHandler( - const std::string & database_name_, + const String & remote_database_name_, + const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, - const std::string & metadata_path_, + const String & metadata_path_, ContextPtr context_, const size_t max_block_size_, bool allow_minimal_ddl_, @@ -50,7 +51,7 @@ public: NameSet fetchRequiredTables(pqxx::connection & connection_); private: - using Storages = std::unordered_map; + using MaterializedStorages = std::unordered_map; bool isPublicationExist(pqxx::work & tx); @@ -72,17 +73,19 @@ private: void consumerFunc(); - NameSet loadFromSnapshot(std::string & snapshot_name, Storages & sync_storages); + StoragePtr loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializePostgreSQL * materialized_storage); - std::unordered_map reloadFromSnapshot(const std::vector> & relation_data); + void reloadFromSnapshot(const std::vector> & relation_data); PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const std::string & table_name); + void replaceMaterializedTable(const String & table_name); + Poco::Logger * log; ContextPtr context; /// Remote database name. - const String database_name; + const String remote_database_name, current_database_name; /// Path for replication metadata. const String metadata_path; @@ -119,10 +122,7 @@ private: bool new_publication_created = false; /// MaterializePostgreSQL tables. Used for managing all operations with its internal nested tables. - Storages storages; - - /// List of nested tables, which is passed to replication consumer. - std::unordered_map nested_storages; + MaterializedStorages materialized_storages; }; } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 72611428326..253a11607b6 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes } static const auto NESTED_TABLE_SUFFIX = "_nested"; +static const auto TMP_SUFFIX = "_tmp"; StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( @@ -65,6 +66,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( replication_handler = std::make_unique( remote_database_name, + table_id_.database_name, connection_info, metadata_path, getContext(), @@ -85,6 +87,16 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( } +/// A temporary clone table might be created for current table in order to update its schema and reload +/// all data in the background while current table will still handle read requests. +StoragePtr StorageMaterializePostgreSQL::createTemporary() const +{ + auto table_id = getStorageID(); + auto new_context = Context::createCopy(context); + return StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX), new_context); +} + + StoragePtr StorageMaterializePostgreSQL::getNested() const { return DatabaseCatalog::instance().getTable(nested_table_id, nested_context); @@ -119,6 +131,95 @@ void StorageMaterializePostgreSQL::setStorageMetadata() } +void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) +{ + const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); + + try + { + InterpreterCreateQuery interpreter(ast_create, nested_context); + interpreter.execute(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + + +std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) +{ + auto new_context = Context::createCopy(from_context); + new_context->makeQueryContext(); + new_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + + return new_context; +} + + +void StorageMaterializePostgreSQL::startup() +{ + if (!is_materialize_postgresql_database) + { + replication_handler->addStorage(remote_table_name, this); + replication_handler->startup(); + } +} + + +void StorageMaterializePostgreSQL::shutdown() +{ + if (replication_handler) + replication_handler->shutdown(); +} + + +void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) +{ + if (replication_handler) + replication_handler->shutdownFinal(); + + auto nested_table = getNested(); + if (nested_table && !is_materialize_postgresql_database) + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, nested_table_id, no_delay); +} + + +NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const +{ + return NamesAndTypesList{ + {"_sign", std::make_shared()}, + {"_version", std::make_shared()} + }; +} + + +Pipe StorageMaterializePostgreSQL::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr context_, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) +{ + if (!nested_loaded) + return Pipe(); + + auto nested_table = getNested(); + + return readFinalFromNestedStorage( + nested_table, + column_names, + metadata_snapshot, + query_info, + context_, + processed_stage, + max_block_size, + num_streams); +} + + std::shared_ptr StorageMaterializePostgreSQL::getMaterializedColumnsDeclaration( const String name, const String type, UInt64 default_value) { @@ -274,95 +375,6 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt } -void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) -{ - const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); - - try - { - InterpreterCreateQuery interpreter(ast_create, nested_context); - interpreter.execute(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - -std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) -{ - auto new_context = Context::createCopy(from_context); - new_context->makeQueryContext(); - new_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - - return new_context; -} - - -void StorageMaterializePostgreSQL::startup() -{ - if (!is_materialize_postgresql_database) - { - replication_handler->addStorage(remote_table_name, this); - replication_handler->startup(); - } -} - - -void StorageMaterializePostgreSQL::shutdown() -{ - if (replication_handler) - replication_handler->shutdown(); -} - - -void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) -{ - if (replication_handler) - replication_handler->shutdownFinal(); - - auto nested_table = getNested(); - if (nested_table && !is_materialize_postgresql_database) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, nested_table_id, no_delay); -} - - -NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const -{ - return NamesAndTypesList{ - {"_sign", std::make_shared()}, - {"_version", std::make_shared()} - }; -} - - -Pipe StorageMaterializePostgreSQL::read( - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - SelectQueryInfo & query_info, - ContextPtr context_, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) -{ - if (!nested_loaded) - return Pipe(); - - auto nested_table = getNested(); - - return readFinalFromNestedStorage( - nested_table, - column_names, - metadata_snapshot, - query_info, - context_, - processed_stage, - max_block_size, - num_streams); -} - - void registerStorageMaterializePostgreSQL(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 079061a69d4..c9523f2fea6 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -53,6 +54,8 @@ public: void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure); + StoragePtr createTemporary() const; + StoragePtr getNested() const; StoragePtr tryGetNested() const; From 14d355a6ac8c9f582c91626832522784609603b9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 May 2021 11:50:29 +0000 Subject: [PATCH 071/105] Fix background update --- .../DatabaseMaterializePostgreSQL.cpp | 5 - .../DatabaseMaterializePostgreSQL.h | 2 - .../PostgreSQLReplicationHandler.cpp | 93 +++++++++++++------ .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 - .../StorageMaterializePostgreSQL.cpp | 21 ++++- .../PostgreSQL/StorageMaterializePostgreSQL.h | 39 ++++++-- .../test.py | 52 +++++++++++ 7 files changed, 166 insertions(+), 48 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 7d493d3dcf3..c3672fc2b73 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -189,11 +189,6 @@ DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( } -void DatabaseMaterializePostgreSQL::renameTable(ContextPtr /* context_ */, const String & /* name */, IDatabase & /* to_database */, const String & /* to_name */, bool /* exchange */, bool /* dictionary */) -{ - throw Exception("MaterializePostgreSQL database does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); -} - } #endif diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index bdfe54ace13..17288be8fb2 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -51,8 +51,6 @@ public: void createTable(ContextPtr context, const String & name, const StoragePtr & table, const ASTPtr & query) override; - void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; - void drop(ContextPtr local_context) override; void shutdown() override; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 6607bb6d3f7..9cd859b9368 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -6,16 +6,24 @@ #include #include #include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { static const auto reschedule_ms = 500; -static const auto TMP_SUFFIX = "_tmp"; PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( @@ -174,7 +182,8 @@ void PostgreSQLReplicationHandler::startSynchronization() } -StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializePostgreSQL * materialized_storage) +StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, const String & table_name, + StorageMaterializePostgreSQL * materialized_storage) { auto tx = std::make_shared(connection->getRef()); @@ -185,17 +194,14 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot /// Already connected to needed database, no need to add it to query. query_str = fmt::format("SELECT * FROM {}", table_name); - /// If table schema has changed, the table stops consuming changed from replication stream. - /// If `allow_minimal_ddl` is true, create a new table in the background, load new table schema - /// and all data from scratch. Then execute REPLACE query with Nested table. - /// This is only allowed for MaterializePostgreSQL database engine. materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); auto nested_storage = materialized_storage->getNested(); - auto insert_context = materialized_storage->getNestedTableContext(); auto insert = std::make_shared(); insert->table_id = nested_storage->getStorageID(); + auto insert_context = materialized_storage->getNestedTableContext(); + InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); @@ -208,6 +214,10 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot materialized_storage->setNestedStatus(true); + nested_storage = materialized_storage->getNested(); + auto nested_table_id = nested_storage->getStorageID(); + LOG_TRACE(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); + return nested_storage; } @@ -405,6 +415,10 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector> & relation_data) { + /// If table schema has changed, the table stops consuming changes from replication stream. + /// If `allow_automatic_update` is true, create a new table in the background, load new table schema + /// and all data from scratch. Then execute REPLACE query. + /// This is only allowed for MaterializePostgreSQL database engine. try { auto replication_connection = postgres::createReplicationConnection(connection_info); @@ -415,15 +429,53 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorlockExclusively(String(), context->getSettingsRef().lock_acquire_timeout); + StoragePtr temp_materialized_storage = materialized_storage->as ()->createTemporary(); + auto from_table_id = materialized_storage->as ()->getNestedStorageID(); + auto to_table_id = temp_materialized_storage->as ()->getNestedStorageID(); + + LOG_TRACE(log, "Starting background update of table {}.{}, uuid {} with table {}.{} uuid {}", + from_table_id.database_name, from_table_id.table_name, toString(from_table_id.uuid), + to_table_id.database_name, to_table_id.table_name, toString(to_table_id.uuid)); + /// This snapshot is valid up to the end of the transaction, which exported it. StoragePtr nested_storage = loadFromSnapshot(snapshot_name, table_name, temp_materialized_storage->as ()); - consumer->updateNested(table_name, nested_storage); - consumer->updateSkipList(table_id, start_lsn); - replaceMaterializedTable(table_name); + auto nested_context = materialized_storage->as ()->getNestedTableContext(); + + to_table_id = nested_storage->getStorageID(); + + auto ast_rename = std::make_shared(); + ASTRenameQuery::Element elem + { + ASTRenameQuery::Table{from_table_id.database_name, from_table_id.table_name}, + ASTRenameQuery::Table{to_table_id.database_name, to_table_id.table_name} + }; + ast_rename->elements.push_back(std::move(elem)); + ast_rename->exchange = true; + + try + { + InterpreterRenameQuery(ast_rename, nested_context).execute(); + + nested_storage = materialized_storage->as ()->getNested(); + materialized_storage->setInMemoryMetadata(nested_storage->getInMemoryMetadata()); + + auto nested_table_id = nested_storage->getStorageID(); + LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); + + consumer->updateNested(table_name, nested_storage); + consumer->updateSkipList(table_id, start_lsn); + + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, to_table_id, true); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } } catch (...) @@ -433,25 +485,6 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector(); - - auto outdated_storage = materialized_storages[table_name]; - auto table_id = outdated_storage->getStorageID(); - - ast_replace->replace_table = true; - - ast_replace->table = table_id.table_name; - ast_replace->database = table_id.database_name; - - ast_replace->as_table = table_id.table_name + TMP_SUFFIX; - ast_replace->as_database = table_id.database_name; - - InterpreterCreateQuery interpreter(ast_replace, context); - interpreter.execute(); -} - } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 76aed35ba80..39167f00579 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -79,8 +79,6 @@ private: PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const std::string & table_name); - void replaceMaterializedTable(const String & table_name); - Poco::Logger * log; ContextPtr context; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 253a11607b6..b82474cf3be 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -53,7 +53,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , replication_settings(std::move(replication_settings_)) , is_materialize_postgresql_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") - , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) + , nested_table_id(StorageID(table_id_.database_name, getNestedTableName(), table_id_.uuid)) , nested_context(makeNestedTableContext(context_->getGlobalContext())) { if (table_id_.uuid == UUIDHelpers::Nil) @@ -87,13 +87,27 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( } +StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( + StoragePtr nested_storage_, ContextPtr context_) + : IStorage(nested_storage_->getStorageID()) + , WithContext(context_->getGlobalContext()) + , is_materialize_postgresql_database(true) + , nested_table_id(nested_storage_->getStorageID()) + , nested_context(makeNestedTableContext(context_->getGlobalContext())) +{ + setInMemoryMetadata(nested_storage_->getInMemoryMetadata()); +} + + /// A temporary clone table might be created for current table in order to update its schema and reload /// all data in the background while current table will still handle read requests. StoragePtr StorageMaterializePostgreSQL::createTemporary() const { auto table_id = getStorageID(); auto new_context = Context::createCopy(context); - return StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX), new_context); + const String temp_storage_name = table_id.table_name + TMP_SUFFIX; + auto temp_storage = StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, temp_storage_name, UUIDHelpers::generateV4()), new_context); + return std::move(temp_storage); } @@ -109,7 +123,7 @@ StoragePtr StorageMaterializePostgreSQL::tryGetNested() const } -std::string StorageMaterializePostgreSQL::getNestedTableName() const +String StorageMaterializePostgreSQL::getNestedTableName() const { auto table_id = getStorageID(); @@ -289,6 +303,7 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt auto table_id = getStorageID(); create_table_query->table = getNestedTableName(); create_table_query->database = table_id.database_name; + create_table_query->uuid = table_id.uuid; auto columns_declare_list = std::make_shared(); auto columns_expression_list = std::make_shared(); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index c9523f2fea6..6bc18d681d4 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -24,14 +24,39 @@ namespace DB { +/** Case of MaterializePostgreSQL database engine. + * There is a table with engine MaterializePostgreSQL. It has a nested table with engine ReplacingMergeTree. + * Both tables shared table_id.table_name and table_id.database_name (probably they automatically have the same uuid?). + * + * MaterializePostgreSQL table does not actually exists only in memory and acts as a wrapper for nested table. + * + * Also it has the same InMemoryMetadata as its nested table, so if metadata of nested table changes - main table also has + * to update its metadata, because all read requests are passed to MaterializePostgreSQL table and then it redirects read + * into nested table. + * + * When there is a need to update table structure, there will be created a new MaterializePostgreSQL table with its own nested table, + * it will have upadated table schema and all data will be loaded from scratch in the background, while previos table with outadted table + * structure will still serve read requests. When data is loaded, a replace query will be done, to swap tables atomically. + * + * In order to update MaterializePostgreSQL table: + * 1. need to update InMemoryMetadata of MaterializePostgreSQL table; + * 2. need to have a new updated ReplacingMergeTree table on disk. + * + * At the point before replace query there are: + * 1. In-memory MaterializePostgreSQL table `databae_name`.`table_name` -- outdated + * 2. On-disk ReplacingMergeTree table with `databae_name`.`table_name` -- outdated + * 3. In-memory MaterializePostgreSQL table `databae_name`.`table_name_tmp` -- updated + * 4. On-disk ReplacingMergeTree table with `databae_name`.`table_name_tmp` -- updated +**/ + class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage, WithContext { friend struct ext::shared_ptr_helper; public: - StorageMaterializePostgreSQL( - const StorageID & table_id_, - ContextPtr context_); + StorageMaterializePostgreSQL(const StorageID & table_id_, ContextPtr context_); + + StorageMaterializePostgreSQL(StoragePtr nested_table, ContextPtr context); String getName() const override { return "MaterializePostgreSQL"; } @@ -70,6 +95,10 @@ public: void renameNested(); + StorageID getNestedStorageID() { return nested_table_id; } + + static std::shared_ptr makeNestedTableContext(ContextPtr from_context); + protected: StorageMaterializePostgreSQL( const StorageID & table_id_, @@ -88,9 +117,7 @@ private: ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure); - std::string getNestedTableName() const; - - static std::shared_ptr makeNestedTableContext(ContextPtr from_context); + String getNestedTableName() const; std::string remote_table_name; std::unique_ptr replication_settings; diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 503e12c890f..91f4c963d30 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -372,6 +372,58 @@ def test_replica_identity_index(started_cluster): check_tables_are_synchronized('postgresql_replica', order_by='key1'); +@pytest.mark.timeout(320) +def test_table_schema_changes(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(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); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) + + instance.query( + """CREATE DATABASE test_database + ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + SETTINGS postgresql_replica_allow_minimal_ddl = 1; + """) + + 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)) + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); + + altered_table = random.randint(0, 4) + cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) + + for i in range(NUM_TABLES): + cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) + cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) + + check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + if i != altered_table: + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) + else: + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) + + check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + cursor.execute('drop table postgresql_replica_{};'.format(i)) + + instance.query("DROP DATABASE test_database") + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 1d243e11e5bdb443551437aebfb52399b7fdfa2f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 May 2021 11:53:20 +0000 Subject: [PATCH 072/105] Update read for materialized --- src/Storages/ReadFinalForExternalReplicaStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index 985b9104085..fb96bb01936 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -71,7 +71,7 @@ Pipe readFinalFromNestedStorage( { Block pipe_header = pipe.getHeader(); auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); - ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true /* add_aliases */, false /* project_result */); pipe.addSimpleTransform([&](const Block & header) { From 01adfb7b3d113de0a893a131e5f66de43f0f60a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 May 2021 13:49:28 +0000 Subject: [PATCH 073/105] Fix single storage case --- src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index b82474cf3be..a0dc6922d1f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -53,7 +53,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , replication_settings(std::move(replication_settings_)) , is_materialize_postgresql_database( DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") - , nested_table_id(StorageID(table_id_.database_name, getNestedTableName(), table_id_.uuid)) + , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) , nested_context(makeNestedTableContext(context_->getGlobalContext())) { if (table_id_.uuid == UUIDHelpers::Nil) @@ -303,7 +303,8 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt auto table_id = getStorageID(); create_table_query->table = getNestedTableName(); create_table_query->database = table_id.database_name; - create_table_query->uuid = table_id.uuid; + if (is_materialize_postgresql_database) + create_table_query->uuid = table_id.uuid; auto columns_declare_list = std::make_shared(); auto columns_expression_list = std::make_shared(); From 3e3396bd9c9e93092d65a90bf47439e63a95735d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 May 2021 09:52:13 +0000 Subject: [PATCH 074/105] Refactor code, add comments --- .../DatabaseMaterializePostgreSQL.cpp | 61 ++++++---- .../DatabaseMaterializePostgreSQL.h | 2 + .../fetchPostgreSQLTableStructure.h | 6 +- src/Storages/IStorage.cpp | 5 +- .../MaterializePostgreSQLConsumer.cpp | 17 ++- .../PostgreSQLReplicationHandler.cpp | 60 +++++----- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 +- .../StorageMaterializePostgreSQL.cpp | 77 ++++++------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 87 ++++++++++----- .../test.py | 105 +++++++++--------- 10 files changed, 237 insertions(+), 187 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index c3672fc2b73..b472e151092 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -69,13 +69,25 @@ void DatabaseMaterializePostgreSQL::startSynchronization() for (const auto & table_name : tables_to_replicate) { - auto storage = tryGetTable(table_name, getContext()); + /// Check nested ReplacingMergeTree table. + auto storage = DatabaseAtomic::tryGetTable(table_name, getContext()); if (!storage) + { + /// Nested table does not exist and will be created by replication thread. storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), getContext()); + } + else + { + /// Nested table was already created and syncronized. + storage = StorageMaterializePostgreSQL::create(storage, getContext()); + } - replication_handler->addStorage(table_name, storage->as()); + /// Cache MaterializePostgreSQL wrapper over nested table. materialized_tables[table_name] = storage; + + /// Let replication thread now, which tables it needs to keep in sync. + replication_handler->addStorage(table_name, storage->as()); } LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); @@ -113,35 +125,36 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte { /// When a nested ReplacingMergeTree table is managed from PostgreSQLReplicationHandler, its context is modified /// to show the type of managed table. - if (local_context->hasQueryContext()) + if ((local_context->hasQueryContext() && local_context->getQueryContext()->getQueryFactoriesInfo().storages.count("ReplacingMergeTree")) + || materialized_tables.empty()) { - auto storage_set = local_context->getQueryContext()->getQueryFactoriesInfo().storages; - if (storage_set.find("ReplacingMergeTree") != storage_set.end()) - { - return DatabaseAtomic::tryGetTable(name, local_context); - } + return DatabaseAtomic::tryGetTable(name, local_context); } /// Note: In select query we call MaterializePostgreSQL table and it calls tryGetTable from its nested. + /// So the only point, where synchronization is needed - access to MaterializePostgreSQL table wrapper over nested table. std::lock_guard lock(tables_mutex); auto table = materialized_tables.find(name); - /// Nested table is not created immediately. Consider that table exists only if nested table exists. - if (table != materialized_tables.end() && table->second->as()->isNestedLoaded()) + /// Return wrapper over ReplacingMergeTree table. If table synchronization just started, table will not + /// be accessible immediately. Table is considered to exist once its nested table was created. + if (table != materialized_tables.end() && table->second->as ()->hasNested()) + { return table->second; + } return StoragePtr{}; } -void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const String & name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) { if (local_context->hasQueryContext()) { auto storage_set = local_context->getQueryContext()->getQueryFactoriesInfo().storages; if (storage_set.find("ReplacingMergeTree") != storage_set.end()) { - DatabaseAtomic::createTable(local_context, name, table, query); + DatabaseAtomic::createTable(local_context, table_name, table, query); return; } } @@ -155,6 +168,15 @@ void DatabaseMaterializePostgreSQL::stopReplication() { if (replication_handler) replication_handler->shutdown(); + + /// Clear wrappers over nested, all access is not done to nested tables directly. + materialized_tables.clear(); +} + + +void DatabaseMaterializePostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) +{ + DatabaseAtomic::dropTable(StorageMaterializePostgreSQL::makeNestedTableContext(local_context), table_name, no_delay); } @@ -169,23 +191,14 @@ void DatabaseMaterializePostgreSQL::drop(ContextPtr local_context) if (metadata.exists()) metadata.remove(false); - DatabaseAtomic::drop(local_context); + DatabaseAtomic::drop(StorageMaterializePostgreSQL::makeNestedTableContext(local_context)); } DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( - ContextPtr /* context */, const DatabaseOnDisk::FilterByNameFunction & /* filter_by_table_name */) + ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) { - Tables nested_tables; - for (const auto & [table_name, storage] : materialized_tables) - { - auto nested_storage = storage->as()->tryGetNested(); - - if (nested_storage) - nested_tables[table_name] = nested_storage; - } - - return std::make_unique(nested_tables, database_name); + return DatabaseAtomic::getTablesIterator(StorageMaterializePostgreSQL::makeNestedTableContext(local_context), filter_by_table_name); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index 17288be8fb2..931ef6836d5 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -51,6 +51,8 @@ public: void createTable(ContextPtr context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + void dropTable(ContextPtr context_, const String & name, bool no_delay) override; + void drop(ContextPtr local_context) override; void shutdown() override; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 2853e0a8ea4..97066f575d8 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -14,9 +14,9 @@ namespace DB struct PostgreSQLTableStructure { - std::shared_ptr columns; - std::shared_ptr primary_key_columns; - std::shared_ptr replica_identity_columns; + std::shared_ptr columns = nullptr; + std::shared_ptr primary_key_columns = nullptr; + std::shared_ptr replica_identity_columns = nullptr; }; using PostgreSQLTableStructurePtr = std::unique_ptr; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index f7fb359432e..2b8ac3a28f0 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -51,7 +51,10 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const std::chron TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout); if (is_dropped) - throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); + { + auto table_id = getStorageID(); + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", table_id.database_name, table_id.table_name); + } return result; } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index bd4e5c0cbe5..7ab6a6f126d 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -308,23 +308,20 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli bool read_next = true; switch (identifier) { - case 'K': - { - /// Only if changed column(s) are part of replica identity index (for now it can be only - /// be primary key - default values for replica identity index). In this case, first comes a tuple - /// with old replica identity indexes and all other values will come as nulls. Then comes a full new row. - readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); - break; - } + /// Only if changed column(s) are part of replica identity index (or primary keys if they are used instead). + /// In this case, first comes a tuple with old replica identity indexes and all other values will come as + /// nulls. Then comes a full new row. + case 'K': [[fallthrough]]; + /// Old row. Only if replica identity is set to full. Does notreally make sense to use it as + /// it is much more efficient to use replica identity index, but support all possible cases. case 'O': { - /// Old row. Only if replica identity is set to full. (For the case when a table does not have any - /// primary key, for now not supported, requires to find suitable order by key(s) for nested table.) readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); break; } case 'N': { + /// New row. readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE); read_next = false; break; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 9cd859b9368..9ad43ddcc94 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -11,14 +11,6 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include namespace DB { @@ -34,7 +26,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( ContextPtr context_, const size_t max_block_size_, bool allow_minimal_ddl_, - bool is_postgresql_replica_database_engine_, + bool is_materialize_postgresql_database_, const String tables_list_) : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , context(context_) @@ -44,7 +36,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , connection_info(connection_info_) , max_block_size(max_block_size_) , allow_minimal_ddl(allow_minimal_ddl_) - , is_postgresql_replica_database_engine(is_postgresql_replica_database_engine_) + , is_materialize_postgresql_database(is_materialize_postgresql_database_) , tables_list(tables_list_) , connection(std::make_shared(connection_info_)) { @@ -128,6 +120,19 @@ void PostgreSQLReplicationHandler::startSynchronization() } }; + /// TODO: think for more cases + bool force_reload = false; + if (is_materialize_postgresql_database) + { + force_reload = !Poco::File(metadata_path).exists(); + } + else + { + assert(materialized_storages.size() == 1); + auto materialized_storage = materialized_storages.begin()->second; + force_reload = !materialized_storage->tryGetNested(); + } + /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. /// Recreation of a replication slot imposes reloading of all tables. @@ -135,7 +140,7 @@ void PostgreSQLReplicationHandler::startSynchronization() { initial_sync(); } - else if (!Poco::File(metadata_path).exists() || new_publication_created) + else if (new_publication_created || force_reload) { /// There are the following cases, which mean that something non-intentioanal happened. /// 1. If replication slot exists and metadata file does not exist, it is not ok. @@ -145,16 +150,15 @@ void PostgreSQLReplicationHandler::startSynchronization() } else { - /// Synchronization and initial load already took place.c + /// Synchronization and initial load already took place. LOG_TRACE(log, "Loading {} tables...", materialized_storages.size()); for (const auto & [table_name, storage] : materialized_storages) { auto materialized_storage = storage->as (); try { - nested_storages[table_name] = materialized_storage->getNested(); - materialized_storage->setStorageMetadata(); - materialized_storage->setNestedStatus(true); + /// Try load nested table, set materialized table metadata. + nested_storages[table_name] = materialized_storage->prepare(); } catch (Exception & e) { @@ -212,9 +216,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); copyData(input, *block_io.out); - materialized_storage->setNestedStatus(true); - - nested_storage = materialized_storage->getNested(); + nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); LOG_TRACE(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); @@ -405,7 +407,7 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( pqxx::ReplicationTransaction & tx, const std::string & table_name) { - if (!is_postgresql_replica_database_engine) + if (!is_materialize_postgresql_database) return nullptr; auto use_nulls = context->getSettingsRef().external_databases_use_nulls; @@ -429,13 +431,15 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras (); + auto table_lock = materialized_storage->lockExclusively(String(), context->getSettingsRef().lock_acquire_timeout); + auto temp_materialized_storage = materialized_storage->createTemporary()->as (); - StoragePtr temp_materialized_storage = materialized_storage->as ()->createTemporary(); - - auto from_table_id = materialized_storage->as ()->getNestedStorageID(); - auto to_table_id = temp_materialized_storage->as ()->getNestedStorageID(); + auto from_table_id = materialized_storage->getNestedStorageID(); + auto to_table_id = temp_materialized_storage->getNestedStorageID(); LOG_TRACE(log, "Starting background update of table {}.{}, uuid {} with table {}.{} uuid {}", from_table_id.database_name, from_table_id.table_name, toString(from_table_id.uuid), @@ -444,8 +448,6 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras ()); - auto nested_context = materialized_storage->as ()->getNestedTableContext(); - to_table_id = nested_storage->getStorageID(); auto ast_rename = std::make_shared(); @@ -457,13 +459,13 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorelements.push_back(std::move(elem)); ast_rename->exchange = true; + auto nested_context = materialized_storage->getNestedTableContext(); + try { InterpreterRenameQuery(ast_rename, nested_context).execute(); - nested_storage = materialized_storage->as ()->getNested(); - materialized_storage->setInMemoryMetadata(nested_storage->getInMemoryMetadata()); - + nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 39167f00579..4a1269f4761 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -34,7 +34,7 @@ public: ContextPtr context_, const size_t max_block_size_, bool allow_minimal_ddl_, - bool is_postgresql_replica_database_engine_, + bool is_materialize_postgresql_database_, const String tables_list = ""); void startup(); @@ -98,7 +98,7 @@ private: bool allow_minimal_ddl = false; /// To distinguish whether current replication handler belongs to a MaterializePostgreSQL database engine or single storage. - bool is_postgresql_replica_database_engine; + bool is_materialize_postgresql_database; /// A coma-separated list of tables, which are going to be replicated for database engine. By default, a whole database is replicated. String tables_list; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index a0dc6922d1f..6b0386ad685 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -39,6 +39,7 @@ static const auto NESTED_TABLE_SUFFIX = "_nested"; static const auto TMP_SUFFIX = "_tmp"; +/// For the case of single storage. StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, const String & remote_database_name, @@ -46,23 +47,23 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, ContextPtr context_, - std::unique_ptr replication_settings_) + std::unique_ptr replication_settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , remote_table_name(remote_table_name_) - , replication_settings(std::move(replication_settings_)) - , is_materialize_postgresql_database( - DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getEngineName() == "MaterializePostgreSQL") - , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) + , is_materialize_postgresql_database(false) + , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) + , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) { if (table_id_.uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializePostgreSQL is allowed only for Atomic database"); setInMemoryMetadata(storage_metadata); + /// Path to store replication metadata (like last written version, etc). auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath() - + "/.metadata_" + table_id_.database_name + "_" + table_id_.table_name; + + "/.metadata_" + table_id_.database_name + "_" + table_id_.table_name + "_" + toString(table_id_.uuid); replication_handler = std::make_unique( remote_database_name, @@ -75,25 +76,30 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( } -StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( - const StorageID & table_id_, - ContextPtr context_) +/// For the case of MaterializePosgreSQL database engine. +/// It is used when nested ReplacingMergeeTree table has not yet be created by replication thread. +/// In this case this storage can't be used for read queries. +StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(const StorageID & table_id_, ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , is_materialize_postgresql_database(true) - , nested_table_id(table_id_) + , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) + , nested_table_id(table_id_) { } -StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( - StoragePtr nested_storage_, ContextPtr context_) +/// Costructor for MaterializePostgreSQL table engine - for the case of MaterializePosgreSQL database engine. +/// It is used when nested ReplacingMergeeTree table has already been created by replication thread. +/// This storage is ready to handle read queries. +StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(StoragePtr nested_storage_, ContextPtr context_) : IStorage(nested_storage_->getStorageID()) , WithContext(context_->getGlobalContext()) , is_materialize_postgresql_database(true) - , nested_table_id(nested_storage_->getStorageID()) + , has_nested(true) , nested_context(makeNestedTableContext(context_->getGlobalContext())) + , nested_table_id(nested_storage_->getStorageID()) { setInMemoryMetadata(nested_storage_->getInMemoryMetadata()); } @@ -105,9 +111,8 @@ StoragePtr StorageMaterializePostgreSQL::createTemporary() const { auto table_id = getStorageID(); auto new_context = Context::createCopy(context); - const String temp_storage_name = table_id.table_name + TMP_SUFFIX; - auto temp_storage = StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, temp_storage_name, UUIDHelpers::generateV4()), new_context); - return std::move(temp_storage); + + return StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX, UUIDHelpers::generateV4()), new_context); } @@ -134,17 +139,6 @@ String StorageMaterializePostgreSQL::getNestedTableName() const } -void StorageMaterializePostgreSQL::setStorageMetadata() -{ - /// If it is a MaterializePostgreSQL database engine, then storage with engine MaterializePostgreSQL - /// gets its metadata when it is fetch from postges, but if inner tables exist (i.e. it is a server restart) - /// then metadata for storage needs to be set from inner table metadata. - auto nested_table = getNested(); - auto storage_metadata = nested_table->getInMemoryMetadataPtr(); - setInMemoryMetadata(*storage_metadata); -} - - void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) { const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); @@ -171,6 +165,15 @@ std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(Co } +StoragePtr StorageMaterializePostgreSQL::prepare() +{ + auto nested_table = getNested(); + setInMemoryMetadata(nested_table->getInMemoryMetadata()); + has_nested.store(true); + return nested_table; +} + + void StorageMaterializePostgreSQL::startup() { if (!is_materialize_postgresql_database) @@ -217,20 +220,18 @@ Pipe StorageMaterializePostgreSQL::read( size_t max_block_size, unsigned num_streams) { - if (!nested_loaded) + /// For database engine there is an invariant: table exists only if its nested table exists, so + /// this check is not needed because read() will never be called until nested is loaded. + /// But for single storage, there is no such invarient. Actually, not sure whether it it better + /// to silently wait until nested is loaded or to throw on read() requests until nested is loaded. + /// TODO: do not use a separate thread in case of single storage, then this problem will be fixed. + if (!has_nested.load()) return Pipe(); + LOG_TRACE(&Poco::Logger::get("kssenii"), "Read method!"); auto nested_table = getNested(); - - return readFinalFromNestedStorage( - nested_table, - column_names, - metadata_snapshot, - query_info, - context_, - processed_stage, - max_block_size, - num_streams); + return readFinalFromNestedStorage(nested_table, column_names, metadata_snapshot, + query_info, context_, processed_stage, max_block_size, num_streams); } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 6bc18d681d4..544fbbe5504 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -24,29 +24,43 @@ namespace DB { +/** Case of single MaterializePostgreSQL table engine. + * + * A user creates a table with engine MaterializePostgreSQL. Order by expression must be specified (needed for + * nested ReplacingMergeTree table). This storage owns its own replication handler, which loads table data + * from PostgreSQL into nested ReplacingMergeTree table. If table is not created, but attached, replication handler + * will not start loading-fron-snapshot procedure, instead it will continue for last commited lsn. + * + * Main point: Both tables exist on disk; database engine interacts only with the main table and main table takes + * total ownershot over nested table. Nested table has name `main_table_uuid` + NESTED_SUFFIX. + * + * TODO: a check is needed for existance of nested, now this case is checked via replication slot existance. +**/ + + /** Case of MaterializePostgreSQL database engine. - * There is a table with engine MaterializePostgreSQL. It has a nested table with engine ReplacingMergeTree. - * Both tables shared table_id.table_name and table_id.database_name (probably they automatically have the same uuid?). * - * MaterializePostgreSQL table does not actually exists only in memory and acts as a wrapper for nested table. + * MaterializePostgreSQL table exists only in memory and acts as a wrapper for nested table, i.e. only provides an + * interface to work with nested table. Both tables share the same StorageID. * - * Also it has the same InMemoryMetadata as its nested table, so if metadata of nested table changes - main table also has + * Main table is never created or droppped via database method. The only way database engine interacts with + * MaterializePostgreSQL table - in tryGetTable() method, a MaterializePostgreSQL table is returned in order to wrap + * and redirect read requests. Set of such wrapper-tables is cached inside database engine. All other methods in + * regard to materializePostgreSQL table are handled by replication handler. + * + * All database methods, apart from tryGetTable(), are devoted only to nested table. + * TODO: It makes sence to allow rename method for MaterializePostgreSQL table via database method. + * TODO: Make sure replication-to-table data channel is done only by relation_id. + * + * Also main table has the same InMemoryMetadata as its nested table, so if metadata of nested table changes - main table also has * to update its metadata, because all read requests are passed to MaterializePostgreSQL table and then it redirects read * into nested table. * * When there is a need to update table structure, there will be created a new MaterializePostgreSQL table with its own nested table, - * it will have upadated table schema and all data will be loaded from scratch in the background, while previos table with outadted table - * structure will still serve read requests. When data is loaded, a replace query will be done, to swap tables atomically. + * it will have updated table schema and all data will be loaded from scratch in the background, while previous table with outadted table + * structure will still serve read requests. When data is loaded, nested tables will be swapped, metadata of metarialzied table will be + * updated according to nested table. * - * In order to update MaterializePostgreSQL table: - * 1. need to update InMemoryMetadata of MaterializePostgreSQL table; - * 2. need to have a new updated ReplacingMergeTree table on disk. - * - * At the point before replace query there are: - * 1. In-memory MaterializePostgreSQL table `databae_name`.`table_name` -- outdated - * 2. On-disk ReplacingMergeTree table with `databae_name`.`table_name` -- outdated - * 3. In-memory MaterializePostgreSQL table `databae_name`.`table_name_tmp` -- updated - * 4. On-disk ReplacingMergeTree table with `databae_name`.`table_name_tmp` -- updated **/ class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper, public IStorage, WithContext @@ -64,6 +78,7 @@ public: void shutdown() override; + /// Used only for single MaterializePostgreSQL storage. void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override; NamesAndTypesList getVirtuals() const override; @@ -77,28 +92,26 @@ public: size_t max_block_size, unsigned num_streams) override; - void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure); + bool hasNested() { return has_nested.load(); } - StoragePtr createTemporary() const; + void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure); StoragePtr getNested() const; StoragePtr tryGetNested() const; + StoragePtr createTemporary() const; + ContextPtr getNestedTableContext() const { return nested_context; } - void setNestedStatus(bool loaded) { nested_loaded.store(loaded); } - - bool isNestedLoaded() { return nested_loaded.load(); } - - void setStorageMetadata(); - void renameNested(); StorageID getNestedStorageID() { return nested_table_id; } static std::shared_ptr makeNestedTableContext(ContextPtr from_context); + StoragePtr prepare(); + protected: StorageMaterializePostgreSQL( const StorageID & table_id_, @@ -107,7 +120,7 @@ protected: const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, ContextPtr context_, - std::unique_ptr replication_settings_); + std::unique_ptr replication_settings); private: static std::shared_ptr getMaterializedColumnsDeclaration( @@ -119,13 +132,31 @@ private: String getNestedTableName() const; - std::string remote_table_name; - std::unique_ptr replication_settings; + /// Needed only for the case of single MaterializePostgreSQL storage - in order to make + /// delayed storage forwarding into replication handler. + String remote_table_name; + + /// Not nullptr only for single MaterializePostgreSQL storage, because for MaterializePostgreSQL + /// database engine there is one replication handler for all tables. std::unique_ptr replication_handler; - std::atomic nested_loaded = false; + + /// Distinguish between single MaterilizePostgreSQL table engine and MaterializePostgreSQL database engine, + /// because table with engine MaterilizePostgreSQL acts differently in each case. bool is_materialize_postgresql_database = false; - StorageID nested_table_id; + + /// Will be set to `true` only once - when nested table was loaded by replication thread. + /// After that, it will never be changed. Needed for MaterializePostgreSQL database engine + /// because there is an invariant - table exists only if its nested table exists, but nested + /// table is not loaded immediately. It is made atomic, because it is accessed only by database engine, + /// and updated by replication handler (only once). + std::atomic has_nested = false; + + /// Nested table context is a copy of global context, but contains query context with defined + /// ReplacingMergeTree storage in factoriesLog. This is needed to let database engine know + /// whether to access nested table or a wrapper over nested (materialized table). ContextPtr nested_context; + + StorageID nested_table_id; }; } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 91f4c963d30..ad27efe33cd 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -93,7 +93,6 @@ def started_cluster(): @pytest.fixture(autouse=True) def postgresql_setup_teardown(): yield # run test - instance.query('DROP TABLE IF EXISTS test.postgresql_replica') @pytest.mark.timeout(120) @@ -372,57 +371,59 @@ def test_replica_identity_index(started_cluster): check_tables_are_synchronized('postgresql_replica', order_by='key1'); -@pytest.mark.timeout(320) -def test_table_schema_changes(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(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); - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) - - instance.query( - """CREATE DATABASE test_database - ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS postgresql_replica_allow_minimal_ddl = 1; - """) - - 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)) - - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - - expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); - - altered_table = random.randint(0, 4) - cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) - - for i in range(NUM_TABLES): - cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) - cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) - - check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - - for i in range(NUM_TABLES): - if i != altered_table: - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) - else: - instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) - - check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - - for i in range(NUM_TABLES): - cursor.execute('drop table postgresql_replica_{};'.format(i)) - - instance.query("DROP DATABASE test_database") - +#@pytest.mark.timeout(320) +#def test_table_schema_changes(started_cluster): +# instance.query("DROP DATABASE IF EXISTS test_database") +# conn = get_postgres_conn(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); +# instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) +# +# instance.query( +# """CREATE DATABASE test_database +# ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') +# SETTINGS postgresql_replica_allow_minimal_ddl = 1; +# """) +# +# 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)) +# +# for i in range(NUM_TABLES): +# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); +# +# expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); +# +# altered_table = random.randint(0, 4) +# cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) +# +# for i in range(NUM_TABLES): +# cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) +# cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) +# +# check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); +# print('check1 OK') +# for i in range(NUM_TABLES): +# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); +# +# for i in range(NUM_TABLES): +# if i != altered_table: +# instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) +# else: +# instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) +# +# check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); +# print('check2 OK') +# for i in range(NUM_TABLES): +# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); +# +# for i in range(NUM_TABLES): +# cursor.execute('drop table postgresql_replica_{};'.format(i)) +# +# instance.query("DROP DATABASE test_database") +# if __name__ == '__main__': cluster.start() From 6a06d725808f5e1609b1372449d61468f19f14ae Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 May 2021 17:28:54 +0000 Subject: [PATCH 075/105] Fix background update --- .../PostgreSQLReplicationHandler.cpp | 46 ++++---- .../StorageMaterializePostgreSQL.cpp | 28 +++-- .../PostgreSQL/StorageMaterializePostgreSQL.h | 6 +- .../test.py | 108 +++++++++--------- 4 files changed, 105 insertions(+), 83 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 9ad43ddcc94..efa081bd00e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -429,32 +429,29 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras (); - auto table_lock = materialized_storage->lockExclusively(String(), context->getSettingsRef().lock_acquire_timeout); - auto temp_materialized_storage = materialized_storage->createTemporary()->as (); - - auto from_table_id = materialized_storage->getNestedStorageID(); - auto to_table_id = temp_materialized_storage->getNestedStorageID(); - - LOG_TRACE(log, "Starting background update of table {}.{}, uuid {} with table {}.{} uuid {}", - from_table_id.database_name, from_table_id.table_name, toString(from_table_id.uuid), - to_table_id.database_name, to_table_id.table_name, toString(to_table_id.uuid)); + auto temp_materialized_storage = materialized_storage->createTemporary(); /// This snapshot is valid up to the end of the transaction, which exported it. - StoragePtr nested_storage = loadFromSnapshot(snapshot_name, table_name, - temp_materialized_storage->as ()); - to_table_id = nested_storage->getStorageID(); + StoragePtr temp_nested_storage = loadFromSnapshot(snapshot_name, table_name, temp_materialized_storage->as ()); + + auto table_id = materialized_storage->getNestedStorageID(); + auto temp_table_id = temp_nested_storage->getStorageID(); + + LOG_TRACE(log, "Starting background update of table {}.{} ({}) with table {}.{} ({})", + table_id.database_name, table_id.table_name, toString(table_id.uuid), + temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); auto ast_rename = std::make_shared(); ASTRenameQuery::Element elem { - ASTRenameQuery::Table{from_table_id.database_name, from_table_id.table_name}, - ASTRenameQuery::Table{to_table_id.database_name, to_table_id.table_name} + ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, + ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} }; ast_rename->elements.push_back(std::move(elem)); ast_rename->exchange = true; @@ -465,14 +462,21 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorprepare(); - auto nested_table_id = nested_storage->getStorageID(); - LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); + { + auto table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); + auto nested_table_id = nested_storage->getStorageID(); - consumer->updateNested(table_name, nested_storage); - consumer->updateSkipList(table_id, start_lsn); + materialized_storage->setNestedStorageID(nested_table_id); + nested_storage = materialized_storage->prepare(); + LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, to_table_id, true); + consumer->updateNested(table_name, nested_storage); + consumer->updateSkipList(relation_id, start_lsn); + } + + LOG_DEBUG(log, "Dropping table {}.{} ({})", temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true); } catch (...) { diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 6b0386ad685..402a2d21ecc 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -54,7 +54,6 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , is_materialize_postgresql_database(false) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) - , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) { if (table_id_.uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializePostgreSQL is allowed only for Atomic database"); @@ -85,7 +84,6 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(const StorageID & tab , is_materialize_postgresql_database(true) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) - , nested_table_id(table_id_) { } @@ -112,19 +110,19 @@ StoragePtr StorageMaterializePostgreSQL::createTemporary() const auto table_id = getStorageID(); auto new_context = Context::createCopy(context); - return StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX, UUIDHelpers::generateV4()), new_context); + return StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX), new_context); } StoragePtr StorageMaterializePostgreSQL::getNested() const { - return DatabaseCatalog::instance().getTable(nested_table_id, nested_context); + return DatabaseCatalog::instance().getTable(getNestedStorageID(), nested_context); } StoragePtr StorageMaterializePostgreSQL::tryGetNested() const { - return DatabaseCatalog::instance().tryGetTable(nested_table_id, nested_context); + return DatabaseCatalog::instance().tryGetTable(getNestedStorageID(), nested_context); } @@ -139,6 +137,17 @@ String StorageMaterializePostgreSQL::getNestedTableName() const } +StorageID StorageMaterializePostgreSQL::getNestedStorageID() const +{ + if (nested_table_id.has_value()) + return nested_table_id.value(); + + auto table_id = getStorageID(); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No storageID found for inner table. ({}.{}, {})", table_id.database_name, table_id.table_name, toString(table_id.uuid)); +} + + void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) { const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); @@ -147,6 +156,12 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure { InterpreterCreateQuery interpreter(ast_create, nested_context); interpreter.execute(); + + auto table_id = getStorageID(); + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); + + /// Save storage_id with correct uuid. + nested_table_id = nested_storage->getStorageID(); } catch (...) { @@ -198,7 +213,7 @@ void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr auto nested_table = getNested(); if (nested_table && !is_materialize_postgresql_database) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, nested_table_id, no_delay); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), no_delay); } @@ -228,7 +243,6 @@ Pipe StorageMaterializePostgreSQL::read( if (!has_nested.load()) return Pipe(); - LOG_TRACE(&Poco::Logger::get("kssenii"), "Read method!"); auto nested_table = getNested(); return readFinalFromNestedStorage(nested_table, column_names, metadata_snapshot, query_info, context_, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 544fbbe5504..6dc405a21f0 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -106,7 +106,9 @@ public: void renameNested(); - StorageID getNestedStorageID() { return nested_table_id; } + StorageID getNestedStorageID() const; + + void setNestedStorageID(const StorageID & id) { nested_table_id.emplace(id); } static std::shared_ptr makeNestedTableContext(ContextPtr from_context); @@ -156,7 +158,7 @@ private: /// whether to access nested table or a wrapper over nested (materialized table). ContextPtr nested_context; - StorageID nested_table_id; + std::optional nested_table_id; }; } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index ad27efe33cd..6f5acb88f1f 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -371,59 +371,61 @@ def test_replica_identity_index(started_cluster): check_tables_are_synchronized('postgresql_replica', order_by='key1'); -#@pytest.mark.timeout(320) -#def test_table_schema_changes(started_cluster): -# instance.query("DROP DATABASE IF EXISTS test_database") -# conn = get_postgres_conn(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); -# instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) -# -# instance.query( -# """CREATE DATABASE test_database -# ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') -# SETTINGS postgresql_replica_allow_minimal_ddl = 1; -# """) -# -# 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)) -# -# for i in range(NUM_TABLES): -# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); -# -# expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); -# -# altered_table = random.randint(0, 4) -# cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) -# -# for i in range(NUM_TABLES): -# cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) -# cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) -# -# check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); -# print('check1 OK') -# for i in range(NUM_TABLES): -# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); -# -# for i in range(NUM_TABLES): -# if i != altered_table: -# instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) -# else: -# instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) -# -# check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); -# print('check2 OK') -# for i in range(NUM_TABLES): -# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); -# -# for i in range(NUM_TABLES): -# cursor.execute('drop table postgresql_replica_{};'.format(i)) -# -# instance.query("DROP DATABASE test_database") -# +@pytest.mark.timeout(320) +def test_table_schema_changes(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(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); + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) + + instance.query( + """CREATE DATABASE test_database + ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + SETTINGS postgresql_replica_allow_minimal_ddl = 1; + """) + + 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)) + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key"); + + altered_table = random.randint(0, 4) + cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table)) + + for i in range(NUM_TABLES): + cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i)) + cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i)) + + assert_nested_table_is_created('postgresql_replica_{}'.format(altered_table)) + check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)) + print('check1 OK') + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + if i != altered_table: + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i)) + else: + instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i)) + + check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table)); + print('check2 OK') + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + cursor.execute('drop table postgresql_replica_{};'.format(i)) + + instance.query("DROP DATABASE test_database") + if __name__ == '__main__': cluster.start() From 19ecdceb6b9ea5f209a7258ea571b95a5cfa8841 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 May 2021 18:38:44 +0000 Subject: [PATCH 076/105] Do not start sync startup in a separate thread in case of single storage --- .../PostgreSQLReplicationHandler.cpp | 10 ++++-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 +-- .../StorageMaterializePostgreSQL.cpp | 32 ++++++++++++------- .../PostgreSQL/StorageMaterializePostgreSQL.h | 13 +++++--- .../test_storage_postgresql_replica/test.py | 9 ++++-- 5 files changed, 45 insertions(+), 23 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index efa081bd00e..82087b3f292 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -66,7 +66,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { /// Will throw pqxx::broken_connection if no connection at the moment connection->get(); - startSynchronization(); + startSynchronization(false); } catch (const pqxx::broken_connection & pqxx_error) { @@ -88,7 +88,7 @@ void PostgreSQLReplicationHandler::shutdown() } -void PostgreSQLReplicationHandler::startSynchronization() +void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { { postgres::Transaction tx(connection->getRef()); @@ -116,6 +116,9 @@ void PostgreSQLReplicationHandler::startSynchronization() { e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); + + if (throw_on_error) + throw; } } }; @@ -164,6 +167,9 @@ void PostgreSQLReplicationHandler::startSynchronization() { e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); + + if (throw_on_error) + throw; } } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 4a1269f4761..2ace4c31198 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -50,6 +50,8 @@ public: /// Fetch list of tables which are going to be replicated. Used for database engine. NameSet fetchRequiredTables(pqxx::connection & connection_); + void startSynchronization(bool throw_on_error); + private: using MaterializedStorages = std::unordered_map; @@ -69,8 +71,6 @@ private: void waitConnectionAndStart(); - void startSynchronization(); - void consumerFunc(); StoragePtr loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializePostgreSQL * materialized_storage); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 402a2d21ecc..ab0e2d4aa0a 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -42,6 +42,7 @@ static const auto TMP_SUFFIX = "_tmp"; /// For the case of single storage. StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const StorageID & table_id_, + bool is_attach_, const String & remote_database_name, const String & remote_table_name_, const postgres::ConnectionInfo & connection_info, @@ -50,10 +51,12 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( std::unique_ptr replication_settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , remote_table_name(remote_table_name_) , is_materialize_postgresql_database(false) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) + , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) + , remote_table_name(remote_table_name_) + , is_attach(is_attach_) { if (table_id_.uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializePostgreSQL is allowed only for Atomic database"); @@ -158,7 +161,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure interpreter.execute(); auto table_id = getStorageID(); - auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, getNestedTableName()), nested_context); /// Save storage_id with correct uuid. nested_table_id = nested_storage->getStorageID(); @@ -194,7 +197,20 @@ void StorageMaterializePostgreSQL::startup() if (!is_materialize_postgresql_database) { replication_handler->addStorage(remote_table_name, this); - replication_handler->startup(); + + if (is_attach) + { + /// In case of attach table use background startup in a separate thread. First wait untill connection is reachable, + /// then check for nested table -- it should already be created. + replication_handler->startup(); + } + else + { + /// Start synchronization preliminary setup immediately and throw in case of failure. + /// It should be guaranteed that if MaterializePostgreSQL table was created successfully, then + /// its nested table was also created. + replication_handler->startSynchronization(/* throw_on_error */ true); + } } } @@ -235,14 +251,6 @@ Pipe StorageMaterializePostgreSQL::read( size_t max_block_size, unsigned num_streams) { - /// For database engine there is an invariant: table exists only if its nested table exists, so - /// this check is not needed because read() will never be called until nested is loaded. - /// But for single storage, there is no such invarient. Actually, not sure whether it it better - /// to silently wait until nested is loaded or to throw on read() requests until nested is loaded. - /// TODO: do not use a separate thread in case of single storage, then this problem will be fixed. - if (!has_nested.load()) - return Pipe(); - auto nested_table = getNested(); return readFinalFromNestedStorage(nested_table, column_names, metadata_snapshot, query_info, context_, processed_stage, max_block_size, num_streams); @@ -453,7 +461,7 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) engine_args[4]->as().value.safeGet()); return StorageMaterializePostgreSQL::create( - args.table_id, remote_database, remote_table, connection_info, + args.table_id, args.attach, remote_database, remote_table, connection_info, metadata, args.getContext(), std::move(postgresql_replication_settings)); }; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 6dc405a21f0..02c758bc95d 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -29,7 +29,7 @@ namespace DB * A user creates a table with engine MaterializePostgreSQL. Order by expression must be specified (needed for * nested ReplacingMergeTree table). This storage owns its own replication handler, which loads table data * from PostgreSQL into nested ReplacingMergeTree table. If table is not created, but attached, replication handler - * will not start loading-fron-snapshot procedure, instead it will continue for last commited lsn. + * will not start loading-from-snapshot procedure, instead it will continue from last commited lsn. * * Main point: Both tables exist on disk; database engine interacts only with the main table and main table takes * total ownershot over nested table. Nested table has name `main_table_uuid` + NESTED_SUFFIX. @@ -117,6 +117,7 @@ public: protected: StorageMaterializePostgreSQL( const StorageID & table_id_, + bool is_attach_, const String & remote_database_name, const String & remote_table_name, const postgres::ConnectionInfo & connection_info, @@ -134,10 +135,6 @@ private: String getNestedTableName() const; - /// Needed only for the case of single MaterializePostgreSQL storage - in order to make - /// delayed storage forwarding into replication handler. - String remote_table_name; - /// Not nullptr only for single MaterializePostgreSQL storage, because for MaterializePostgreSQL /// database engine there is one replication handler for all tables. std::unique_ptr replication_handler; @@ -159,6 +156,12 @@ private: ContextPtr nested_context; std::optional nested_table_id; + + /// Needed only for the case of single MaterializePostgreSQL storage - in order to make + /// delayed storage forwarding into replication handler. + String remote_table_name; + + bool is_attach; }; } diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 678c7384c1d..ed61dcba935 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -100,7 +100,6 @@ def test_no_connection_at_startup(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") - started_cluster.pause_container('postgres1') instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) ENGINE = MaterializePostgreSQL( @@ -108,6 +107,12 @@ def test_no_connection_at_startup(started_cluster): PRIMARY KEY key; ''') time.sleep(3) + + instance.query('DETACH TABLE test.postgresql_replica') + started_cluster.pause_container('postgres1') + + instance.query('ATTACH TABLE test.postgresql_replica') + time.sleep(3) started_cluster.unpause_container('postgres1') result = instance.query('SELECT count() FROM test.postgresql_replica;') @@ -325,7 +330,7 @@ def test_many_replication_messages(started_cluster): ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') - while (int(result) == 100000): + while (int(result) != 100000): time.sleep(0.2) result = instance.query('SELECT count() FROM test.postgresql_replica;') print("SYNC OK") From eff26f9d5451a40ba857009855f78eb43cff2aba Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 May 2021 21:42:06 +0000 Subject: [PATCH 077/105] Add comments, small improvements --- .../DatabaseMaterializePostgreSQL.cpp | 19 ++++-- .../MaterializePostgreSQLConsumer.cpp | 66 +++++++++++++------ .../MaterializePostgreSQLConsumer.h | 11 ++-- .../MaterializePostgreSQLSettings.cpp | 2 +- .../MaterializePostgreSQLSettings.h | 10 +-- .../PostgreSQLReplicationHandler.cpp | 47 ++++++------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 15 ++++- .../StorageMaterializePostgreSQL.cpp | 5 +- .../PostgreSQL/StorageMaterializePostgreSQL.h | 17 ++++- .../test.py | 4 +- .../test_storage_postgresql_replica/test.py | 2 +- 11 files changed, 128 insertions(+), 70 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index b472e151092..f9cc9aa5ce2 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static const auto METADATA_SUFFIX = ".postgresql_replica_metadata"; +static const auto METADATA_SUFFIX = ".materialize_postgresql_metadata"; DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( ContextPtr context_, @@ -60,10 +60,10 @@ void DatabaseMaterializePostgreSQL::startSynchronization() connection->getConnectionInfo(), metadata_path + METADATA_SUFFIX, getContext(), - settings->postgresql_replica_max_block_size.value, - settings->postgresql_replica_allow_minimal_ddl, + settings->materialize_postgresql_max_block_size.value, + settings->materialize_postgresql_allow_automatic_update, /* is_materialize_postgresql_database = */ true, - settings->postgresql_replica_tables_list.value); + settings->materialize_postgresql_tables_list.value); std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection->getRef()); @@ -90,7 +90,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() replication_handler->addStorage(table_name, storage->as()); } - LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); + LOG_TRACE(log, "Loaded {} tables. Starting synchronization, (database: {})", materialized_tables.size(), database_name); replication_handler->startup(); } @@ -123,8 +123,10 @@ void DatabaseMaterializePostgreSQL::loadStoredObjects(ContextPtr local_context, StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, ContextPtr local_context) const { - /// When a nested ReplacingMergeTree table is managed from PostgreSQLReplicationHandler, its context is modified - /// to show the type of managed table. + /// In otder to define which table access is needed - to MaterializePostgreSQL table (only in case of SELECT queries) or + /// to its nested ReplacingMergeTree table (in all other cases), the context of a query os modified. + /// Also if materialzied_tables set is empty - it means all access is done to ReplacingMergeTree tables - it is a case after + /// replication_handler was shutdown. if ((local_context->hasQueryContext() && local_context->getQueryContext()->getQueryFactoriesInfo().storages.count("ReplacingMergeTree")) || materialized_tables.empty()) { @@ -149,6 +151,7 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) { + /// Create table query can only be called from replication thread. if (local_context->hasQueryContext()) { auto storage_set = local_context->getQueryContext()->getQueryFactoriesInfo().storages; @@ -176,6 +179,7 @@ void DatabaseMaterializePostgreSQL::stopReplication() void DatabaseMaterializePostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) { + /// Modify context into nested_context and pass query to Atomic database. DatabaseAtomic::dropTable(StorageMaterializePostgreSQL::makeNestedTableContext(local_context), table_name, no_delay); } @@ -198,6 +202,7 @@ void DatabaseMaterializePostgreSQL::drop(ContextPtr local_context) DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) { + /// Modify context into nested_context and pass query to Atomic database. return DatabaseAtomic::getTablesIterator(StorageMaterializePostgreSQL::makeNestedTableContext(local_context), filter_by_table_name); } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 7ab6a6f126d..74a6419ac5d 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -30,7 +30,7 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, - bool allow_minimal_ddl_, + bool allow_automatic_update_, Storages storages_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) , context(context_) @@ -40,7 +40,7 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( , connection(std::move(connection_)) , current_lsn(start_lsn) , max_block_size(max_block_size_) - , allow_minimal_ddl(allow_minimal_ddl_) + , allow_automatic_update(allow_automatic_update_) , storages(storages_) { for (const auto & [table_name, storage] : storages) @@ -218,10 +218,13 @@ void MaterializePostgreSQLConsumer::readTupleData( break; } case 'u': /// TOAST value && unchanged at the same time. Actual value is not sent. + { /// TOAST values are not supported. (TOAST values are values that are considered in postgres /// to be too large to be stored directly) + LOG_WARNING(log, "Got TOAST value, which is not supported, default value will be used instead."); insertDefaultValue(buffer, column_idx); break; + } } }; @@ -536,13 +539,20 @@ String MaterializePostgreSQLConsumer::advanceLSN(std::shared_ptrsecond; + + /// Table is in a skip list and has not yet received a valid lsn == it has not been reloaded. if (table_start_lsn.empty()) return false; + /// Table has received a valid lsn, but it is not yet at a position, from which synchronization is + /// allowed. It is allowed only after lsn position, returned with snapshot, from which + /// table was reloaded. if (getLSNValue(current_lsn) >= getLSNValue(table_start_lsn)) { LOG_TRACE(log, "Synchronization is resumed for table: {} (start_lsn: {})", @@ -559,14 +569,21 @@ bool MaterializePostgreSQLConsumer::isSyncAllowed(Int32 relation_id) void MaterializePostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name) { + /// Empty lsn string means - continue wating for valid lsn. skip_list.insert({relation_id, ""}); + + /// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream + /// and it receives first data after update. schema_data.erase(relation_id); + + /// Clear table buffer. auto & buffer = buffers.find(relation_name)->second; buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); - if (!allow_minimal_ddl) - LOG_WARNING(log, "Table {} is skipped, because table schema has changed", relation_name); + + if (allow_automatic_update) + LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id); else - LOG_TRACE(log, "Table {} is skipped temporarily. ID: {}", relation_name, relation_id); + LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name); } @@ -646,37 +663,48 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() bool MaterializePostgreSQLConsumer::consume(std::vector> & skipped_tables) { + /// Check if there are tables, which are skipped from being updated by changes from replication stream, + /// because schema changes were detected. Update them, if it is allowed. + if (allow_automatic_update && !skip_list.empty()) + { + for (const auto & [relation_id, lsn] : skip_list) + { + /// Non-empty lsn in this place means that table was already updated, but no changes for that table were + /// received in a previous stream. A table is removed from skip list only when there came + /// changes for table with lsn higher than lsn of snapshot, from which table was reloaded. Since table + /// reaload and reading from replication stream are done in the same thread, no lsn will be skipped + /// between these two events. + if (lsn.empty()) + skipped_tables.emplace_back(std::make_pair(relation_id, relation_id_to_name[relation_id])); + } + } + + /// Read up to max_block_size changed (approximately - in same cases might be more). if (!readFromReplicationSlot()) { - if (allow_minimal_ddl && !skip_list.empty()) - { - for (const auto & [relation_id, lsn] : skip_list) - { - if (lsn.empty()) - skipped_tables.emplace_back(std::make_pair(relation_id, relation_id_to_name[relation_id])); - } - } - + /// No data was read, reschedule. return false; } + /// Some data was read, schedule as soon as possible. return true; } -void MaterializePostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage) +void MaterializePostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn) { + /// Cache new pointer to replacingMergeTree table. storages[table_name] = nested_storage; + + /// Create a new empty buffer (with updated metadata), where data is first loaded before syncing into actual table. auto & buffer = buffers.find(table_name)->second; buffer.createEmptyBuffer(nested_storage); -} - -void MaterializePostgreSQLConsumer::updateSkipList(Int32 table_id, const String & table_start_lsn) -{ + /// Set start position to valid lsn. Before it was an empty string. Futher read for table allowed, if it has a valid lsn. skip_list[table_id] = table_start_lsn; } + } #endif diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 3bef0c717ba..5bf8c8c7755 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -34,18 +34,19 @@ public: const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, - bool allow_minimal_ddl_, + bool allow_automatic_update_, Storages storages_); void readMetadata(); bool consume(std::vector> & skipped_tables); - void updateNested(const String & table_name, StoragePtr nested_storage); - - void updateSkipList(Int32 table_id, const String & table_start_lsn); + /// Called from reloadFromSnapshot by replication handler. This method is needed to move a table back into synchronization + /// process if it was skipped due to schema changes. + void updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn); private: + /// Read approximarely up to max_block_size changes from WAL. bool readFromReplicationSlot(); void syncTables(std::shared_ptr tx); @@ -109,7 +110,7 @@ private: std::string current_lsn, final_lsn; const size_t max_block_size; - bool allow_minimal_ddl; + bool allow_automatic_update; std::string table_to_insert; diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp index 48fe61b4182..2682bd6194f 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) void MaterializePostgreSQLSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h index a2ad76ceaeb..8875c45f9fa 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h @@ -13,12 +13,12 @@ namespace DB class ASTStorage; -#define LIST_OF_POSTGRESQL_REPLICA_SETTINGS(M) \ - M(UInt64, postgresql_replica_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \ - M(String, postgresql_replica_tables_list, "", "List of tables for MaterializePostgreSQL database engine", 0) \ - M(Bool, postgresql_replica_allow_minimal_ddl, 0, "Allow to track minimal possible ddl. By default, table after ddl will get into a skip list", 0) \ +#define LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS(M) \ + M(UInt64, materialize_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \ + M(String, materialize_postgresql_tables_list, "", "List of tables for MaterializePostgreSQL database engine", 0) \ + M(Bool, materialize_postgresql_allow_automatic_update, 0, "Allow to reload table in the background, when schema changes are detected", 0) \ -DECLARE_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_POSTGRESQL_REPLICA_SETTINGS) +DECLARE_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) struct MaterializePostgreSQLSettings : public BaseSettings { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 82087b3f292..1675bf5870f 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -25,7 +25,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const std::string & metadata_path_, ContextPtr context_, const size_t max_block_size_, - bool allow_minimal_ddl_, + bool allow_automatic_update_, bool is_materialize_postgresql_database_, const String tables_list_) : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) @@ -35,7 +35,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , metadata_path(metadata_path_) , connection_info(connection_info_) , max_block_size(max_block_size_) - , allow_minimal_ddl(allow_minimal_ddl_) + , allow_automatic_update(allow_automatic_update_) , is_materialize_postgresql_database(is_materialize_postgresql_database_) , tables_list(tables_list_) , connection(std::make_shared(connection_info_)) @@ -123,19 +123,6 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } }; - /// TODO: think for more cases - bool force_reload = false; - if (is_materialize_postgresql_database) - { - force_reload = !Poco::File(metadata_path).exists(); - } - else - { - assert(materialized_storages.size() == 1); - auto materialized_storage = materialized_storages.begin()->second; - force_reload = !materialized_storage->tryGetNested(); - } - /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. /// Recreation of a replication slot imposes reloading of all tables. @@ -143,11 +130,10 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { initial_sync(); } - else if (new_publication_created || force_reload) + else if (new_publication_created) { - /// There are the following cases, which mean that something non-intentioanal happened. - /// 1. If replication slot exists and metadata file does not exist, it is not ok. - /// 2. If replication slot exists before publication is created. + /// Replication slot depends on publication, so if replication slot exists and new + /// publication was just created - drop that replication slot and start from scratch. dropReplicationSlot(tx.getRef()); initial_sync(); } @@ -165,6 +151,21 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } catch (Exception & e) { + if (e.code() == ErrorCodes::UNKNOWN_TABLE) + { + try + { + /// If nested table does not exist, try load it once again. + loadFromSnapshot(snapshot_name, table_name, storage->as ()); + nested_storages[table_name] = materialized_storage->prepare(); + continue; + } + catch (Exception & e) + { + e.addMessage("Table load failed for the second time"); + } + } + e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); @@ -182,7 +183,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) metadata_path, start_lsn, max_block_size, - allow_minimal_ddl, + allow_automatic_update, nested_storages); consumer_task->activateAndSchedule(); @@ -469,16 +470,16 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorlockForShare(String(), context->getSettingsRef().lock_acquire_timeout); auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); + auto table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); auto nested_table_id = nested_storage->getStorageID(); materialized_storage->setNestedStorageID(nested_table_id); nested_storage = materialized_storage->prepare(); LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); - consumer->updateNested(table_name, nested_storage); - consumer->updateSkipList(relation_id, start_lsn); + /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. + consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); } LOG_DEBUG(log, "Dropping table {}.{} ({})", temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 2ace4c31198..c337f354247 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -33,10 +33,11 @@ public: const String & metadata_path_, ContextPtr context_, const size_t max_block_size_, - bool allow_minimal_ddl_, + bool allow_automatic_update_, bool is_materialize_postgresql_database_, const String tables_list = ""); + /// Activate task to be run from a separate thread: wait untill connection is available and call startReplication(). void startup(); /// Stop replication without cleanup. @@ -45,16 +46,20 @@ public: /// Clean up replication: remove publication and replication slots. void shutdownFinal(); + /// Add storage pointer to let handler know which tables it needs to keep in sync. void addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage); /// Fetch list of tables which are going to be replicated. Used for database engine. NameSet fetchRequiredTables(pqxx::connection & connection_); + /// Start replication setup immediately. void startSynchronization(bool throw_on_error); private: using MaterializedStorages = std::unordered_map; + /// Methods to manage Publication. + bool isPublicationExist(pqxx::work & tx); void createPublicationIfNeeded(pqxx::work & tx, bool create_without_check = false); @@ -63,12 +68,16 @@ private: void dropPublication(pqxx::nontransaction & ntx); + /// Methods to manage Replication Slots. + bool isReplicationSlotExist(pqxx::nontransaction & tx, std::string & slot_name); void createReplicationSlot(pqxx::nontransaction & tx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); void dropReplicationSlot(pqxx::nontransaction & tx, bool temporary = false); + /// Methods to manage replication. + void waitConnectionAndStart(); void consumerFunc(); @@ -82,7 +91,6 @@ private: Poco::Logger * log; ContextPtr context; - /// Remote database name. const String remote_database_name, current_database_name; /// Path for replication metadata. @@ -95,7 +103,8 @@ private: const size_t max_block_size; /// Table structure changes are always tracked. By default, table with changed schema will get into a skip list. - bool allow_minimal_ddl = false; + /// This setting allows to reloas table in the background. + bool allow_automatic_update = false; /// To distinguish whether current replication handler belongs to a MaterializePostgreSQL database engine or single storage. bool is_materialize_postgresql_database; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index ab0e2d4aa0a..11a44f7d022 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -24,6 +24,7 @@ #include #include +/// TODO: Add test for allow_automatic_update setting in case of single storage. namespace DB { @@ -73,8 +74,8 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( connection_info, metadata_path, getContext(), - replication_settings->postgresql_replica_max_block_size.value, - replication_settings->postgresql_replica_allow_minimal_ddl.value, false); + replication_settings->materialize_postgresql_max_block_size.value, + replication_settings->materialize_postgresql_allow_automatic_update.value, false); } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 02c758bc95d..470ea81cb25 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -92,6 +92,9 @@ public: size_t max_block_size, unsigned num_streams) override; + /// This method is called only from MateriaizePostgreSQL database engine, because it needs to maintain + /// an invariant: a table exists only if its nested table exists. This atomic variable is set to _true_ + /// only once - when nested table is successfully created and is never changed afterwards. bool hasNested() { return has_nested.load(); } void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure); @@ -100,18 +103,21 @@ public: StoragePtr tryGetNested() const; + /// Create a temporary MaterializePostgreSQL table with current_table_name + TMP_SUFFIX. + /// An empty wrapper is returned - it does not have inMemory metadata, just acts as an empty wrapper over + /// temporary nested, which will be created shortly after. StoragePtr createTemporary() const; ContextPtr getNestedTableContext() const { return nested_context; } - void renameNested(); - StorageID getNestedStorageID() const; void setNestedStorageID(const StorageID & id) { nested_table_id.emplace(id); } static std::shared_ptr makeNestedTableContext(ContextPtr from_context); + /// Get nested table (or throw if it does not exist), set in-memory metadata (taken from nested table) + /// for current table, set has_nested = true. StoragePtr prepare(); protected: @@ -155,12 +161,19 @@ private: /// whether to access nested table or a wrapper over nested (materialized table). ContextPtr nested_context; + /// Save nested storageID to be able to fetch it. It is set once nested is created and will be + /// updated only when nested is reloaded or renamed. std::optional nested_table_id; /// Needed only for the case of single MaterializePostgreSQL storage - in order to make /// delayed storage forwarding into replication handler. String remote_table_name; + /// Needed only for the case of single MaterializePostgreSQL storage, because in case of create + /// query (not attach) initial setup wiil be done immediately and error message is thrown at once. + /// It results in the fact: single MaterializePostgreSQL storage is created only if its nested table is created. + /// In case of attach - this setup will be done in a separate thread in the background. It will also + /// be checked for nested table and attempted to load it if it does not exist for some reason. bool is_attach; }; diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 6f5acb88f1f..abfb3427c99 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -270,7 +270,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query(''' CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS postgresql_replica_tables_list = '{}'; + SETTINGS materialize_postgresql_tables_list = '{}'; '''.format(publication_tables)) assert 'test_database' in instance.query('SHOW DATABASES') @@ -385,7 +385,7 @@ def test_table_schema_changes(started_cluster): instance.query( """CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS postgresql_replica_allow_minimal_ddl = 1; + SETTINGS materialize_postgresql_allow_automatic_update = 1; """) for i in range(NUM_TABLES): diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index ed61dcba935..20d21008629 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -326,7 +326,7 @@ def test_many_replication_messages(started_cluster): PRIMARY KEY(key)) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - SETTINGS postgresql_replica_max_block_size = 50000; + SETTINGS materialize_postgresql_max_block_size = 50000; ''') result = instance.query('SELECT count() FROM test.postgresql_replica;') From 89144ba8438aec054a408a1249925fe33e08b694 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 May 2021 22:06:21 +0000 Subject: [PATCH 078/105] Try adding lock for storage --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 3 ++- src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 1675bf5870f..5b6e0ceb16c 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -467,11 +467,12 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorlockForShare(String(), context->getSettingsRef().lock_acquire_timeout); InterpreterRenameQuery(ast_rename, nested_context).execute(); { auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); - auto table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); auto nested_table_id = nested_storage->getStorageID(); materialized_storage->setNestedStorageID(nested_table_id); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 11a44f7d022..38d4790a826 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -24,7 +24,6 @@ #include #include -/// TODO: Add test for allow_automatic_update setting in case of single storage. namespace DB { @@ -75,7 +74,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( metadata_path, getContext(), replication_settings->materialize_postgresql_max_block_size.value, - replication_settings->materialize_postgresql_allow_automatic_update.value, false); + /* allow_automatic_update */ false, /* is_materialize_postgresql_database */false); } @@ -252,6 +251,7 @@ Pipe StorageMaterializePostgreSQL::read( size_t max_block_size, unsigned num_streams) { + auto materialized_table_lock = lockForShare(String(), context_->getSettingsRef().lock_acquire_timeout); auto nested_table = getNested(); return readFinalFromNestedStorage(nested_table, column_names, metadata_snapshot, query_info, context_, processed_stage, max_block_size, num_streams); From ed5c2321cc0b542e379371a61f781c3deb452ba1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 May 2021 10:43:21 +0000 Subject: [PATCH 079/105] Some fixes --- .../materialize-postgresql.md | 31 +++++++++++++++++-- .../integrations/materialize-postgresql.md | 24 ++++++++++++-- .../DatabaseMaterializePostgreSQL.cpp | 2 +- .../DatabaseMaterializePostgreSQL.h | 2 +- .../MaterializePostgreSQLConsumer.cpp | 30 +++++++++--------- .../PostgreSQLReplicationHandler.cpp | 15 ++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- .../StorageMaterializePostgreSQL.cpp | 4 +-- .../PostgreSQL/StorageMaterializePostgreSQL.h | 11 +++---- 9 files changed, 86 insertions(+), 35 deletions(-) diff --git a/docs/en/engines/database-engines/materialize-postgresql.md b/docs/en/engines/database-engines/materialize-postgresql.md index 5f1ee614704..f657035d050 100644 --- a/docs/en/engines/database-engines/materialize-postgresql.md +++ b/docs/en/engines/database-engines/materialize-postgresql.md @@ -7,8 +7,36 @@ toc_title: MaterializePostgreSQL ## Creating a Database {#creating-a-database} +``` sql +CREATE DATABASE test_database +ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' + +SELECT * FROM test_database.postgres_table; +``` + + +## Settings {#settings} + +1. `materialize_postgresql_max_block_size` - Number of rows collected before flushing data into table. Default: `65536`. + +2. `materialize_postgresql_tables_list` - List of tables for MaterializePostgreSQL database engine. Default: `whole database`. + +3. `materialize_postgresql_allow_automatic_update` - Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). + +``` sql +CREATE DATABASE test_database +ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' +SETTINGS materialize_postgresql_max_block_size = 65536, + materialize_postgresql_tables_list = 'table1,table2,table3'; + +SELECT * FROM test_database.table1; +``` + + ## Requirements {#requirements} +- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. + - Each replicated table must have one of the following **replica identity**: 1. **default** (primary key) @@ -36,6 +64,3 @@ postgres# SELECT CASE relreplident FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` - -- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. - diff --git a/docs/en/engines/table-engines/integrations/materialize-postgresql.md b/docs/en/engines/table-engines/integrations/materialize-postgresql.md index e3cbfbb087b..c40ea6b72db 100644 --- a/docs/en/engines/table-engines/integrations/materialize-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialize-postgresql.md @@ -7,10 +7,30 @@ toc_title: MateriaziePostgreSQL ## Creating a Table {#creating-a-table} +``` sql +CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) +ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') +PRIMARY KEY key; +``` + + ## Requirements {#requirements} -- A table with engine `MaterializePostgreSQL` must have a primary key - the same as a replica identity index of a postgres table (See [details on replica identity index](../../database-engines/materialize-postgresql.md#requirements)). +- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. + +- A table with engine `MaterializePostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialize-postgresql.md#requirements)). - Only database `Atomic` is allowed. -- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. + +## Virtual columns {#creating-a-table} + +- `_version` + +- `_sign` + +``` sql +CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) +ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') +PRIMARY KEY key; +``` diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index f9cc9aa5ce2..d954c7ebe7f 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -79,7 +79,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() } else { - /// Nested table was already created and syncronized. + /// Nested table was already created and synchronized. storage = StorageMaterializePostgreSQL::create(storage, getContext()); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index 931ef6836d5..c04564fccbb 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -51,7 +51,7 @@ public: void createTable(ContextPtr context, const String & name, const StoragePtr & table, const ASTPtr & query) override; - void dropTable(ContextPtr context_, const String & name, bool no_delay) override; + void dropTable(ContextPtr local_context, const String & name, bool no_delay) override; void drop(ContextPtr local_context) override; diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 74a6419ac5d..664bdde7d70 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -53,12 +53,17 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage) { const auto storage_metadata = storage->getInMemoryMetadataPtr(); - description.init(storage_metadata->getSampleBlock()); + const Block sample_block = storage_metadata->getSampleBlock(); + description.init(sample_block); columns = description.sample_block.cloneEmptyColumns(); const auto & storage_columns = storage_metadata->getColumns().getAllPhysical(); auto insert_columns = std::make_shared(); + auto table_id = storage->getStorageID(); + LOG_TRACE(&Poco::Logger::get("MaterializePostgreSQLBuffer"), "New buffer for table {}.{} ({}), structure: {}", + table_id.database_name, table_id.table_name, toString(table_id.uuid), sample_block.dumpStructure()); + assert(description.sample_block.columns() == storage_columns.size()); size_t idx = 0; @@ -158,7 +163,7 @@ T MaterializePostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) { - assert(size > pos + 16); + assert(size >= pos + 16); Int64 result = unhexN(message, pos, 8); pos += 16; return result; @@ -167,7 +172,7 @@ Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & po Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) { - assert(size > pos + 8); + assert(size >= pos + 8); Int32 result = unhexN(message, pos, 4); pos += 8; return result; @@ -176,7 +181,7 @@ Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & po Int16 MaterializePostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) { - assert(size > pos + 4); + assert(size >= pos + 4); Int16 result = unhexN(message, pos, 2); pos += 4; return result; @@ -185,7 +190,7 @@ Int16 MaterializePostgreSQLConsumer::readInt16(const char * message, size_t & po Int8 MaterializePostgreSQLConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) { - assert(size > pos + 2); + assert(size >= pos + 2); Int8 result = unhex2(message + pos); pos += 2; return result; @@ -569,7 +574,7 @@ bool MaterializePostgreSQLConsumer::isSyncAllowed(Int32 relation_id) void MaterializePostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name) { - /// Empty lsn string means - continue wating for valid lsn. + /// Empty lsn string means - continue waiting for valid lsn. skip_list.insert({relation_id, ""}); /// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream @@ -680,14 +685,9 @@ bool MaterializePostgreSQLConsumer::consume(std::vector } /// Read up to max_block_size changed (approximately - in same cases might be more). - if (!readFromReplicationSlot()) - { - /// No data was read, reschedule. - return false; - } - - /// Some data was read, schedule as soon as possible. - return true; + /// false: no data was read, reschedule. + /// true: some data was read, schedule as soon as possible. + return readFromReplicationSlot(); } @@ -700,7 +700,7 @@ void MaterializePostgreSQLConsumer::updateNested(const String & table_name, Stor auto & buffer = buffers.find(table_name)->second; buffer.createEmptyBuffer(nested_storage); - /// Set start position to valid lsn. Before it was an empty string. Futher read for table allowed, if it has a valid lsn. + /// Set start position to valid lsn. Before it was an empty string. Further read for table allowed, if it has a valid lsn. skip_list[table_id] = table_start_lsn; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 5b6e0ceb16c..179764564f2 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -17,6 +17,10 @@ namespace DB static const auto reschedule_ms = 500; +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; +} PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const String & remote_database_name_, @@ -143,7 +147,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) LOG_TRACE(log, "Loading {} tables...", materialized_storages.size()); for (const auto & [table_name, storage] : materialized_storages) { - auto materialized_storage = storage->as (); + auto * materialized_storage = storage->as (); try { /// Try load nested table, set materialized table metadata. @@ -160,7 +164,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) nested_storages[table_name] = materialized_storage->prepare(); continue; } - catch (Exception & e) + catch (...) { e.addMessage("Table load failed for the second time"); } @@ -438,9 +442,10 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras (); + context); + auto * materialized_storage = storage->as (); auto temp_materialized_storage = materialized_storage->createTemporary(); @@ -485,6 +490,8 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorstartup(); } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 470ea81cb25..6b896c24dfa 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -29,12 +29,11 @@ namespace DB * A user creates a table with engine MaterializePostgreSQL. Order by expression must be specified (needed for * nested ReplacingMergeTree table). This storage owns its own replication handler, which loads table data * from PostgreSQL into nested ReplacingMergeTree table. If table is not created, but attached, replication handler - * will not start loading-from-snapshot procedure, instead it will continue from last commited lsn. + * will not start loading-from-snapshot procedure, instead it will continue from last committed lsn. * * Main point: Both tables exist on disk; database engine interacts only with the main table and main table takes * total ownershot over nested table. Nested table has name `main_table_uuid` + NESTED_SUFFIX. * - * TODO: a check is needed for existance of nested, now this case is checked via replication slot existance. **/ @@ -43,13 +42,13 @@ namespace DB * MaterializePostgreSQL table exists only in memory and acts as a wrapper for nested table, i.e. only provides an * interface to work with nested table. Both tables share the same StorageID. * - * Main table is never created or droppped via database method. The only way database engine interacts with + * Main table is never created or dropped via database method. The only way database engine interacts with * MaterializePostgreSQL table - in tryGetTable() method, a MaterializePostgreSQL table is returned in order to wrap * and redirect read requests. Set of such wrapper-tables is cached inside database engine. All other methods in * regard to materializePostgreSQL table are handled by replication handler. * * All database methods, apart from tryGetTable(), are devoted only to nested table. - * TODO: It makes sence to allow rename method for MaterializePostgreSQL table via database method. + * NOTE: It makes sense to allow rename method for MaterializePostgreSQL table via database method. * TODO: Make sure replication-to-table data channel is done only by relation_id. * * Also main table has the same InMemoryMetadata as its nested table, so if metadata of nested table changes - main table also has @@ -70,7 +69,7 @@ class StorageMaterializePostgreSQL final : public ext::shared_ptr_helper Date: Sat, 8 May 2021 14:55:53 +0000 Subject: [PATCH 080/105] Correct merge, finish refactoring --- src/Core/PostgreSQL/Connection.cpp | 35 ++++ src/Core/PostgreSQL/Connection.h | 30 ++++ src/Core/PostgreSQL/ConnectionHolder.h | 37 +++++ src/Core/PostgreSQL/PoolWithFailover.cpp | 138 ++++++++++++++++ src/Core/PostgreSQL/PoolWithFailover.h | 65 ++++++++ src/Core/PostgreSQL/Utils.cpp | 19 +++ src/Core/PostgreSQL/Utils.h | 37 +++++ .../MaterializePostgreSQLConsumer.cpp | 8 +- .../MaterializePostgreSQLConsumer.h | 4 +- .../PostgreSQLReplicationHandler.cpp | 156 +++++++++--------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 3 +- .../PostgreSQL/StorageMaterializePostgreSQL.h | 2 +- .../TableFunctionPostgreSQL.cpp | 4 +- src/TableFunctions/TableFunctionPostgreSQL.h | 2 +- .../__init__.py | 0 .../configs/config.xml | 30 ++++ .../configs/users.xml | 23 +++ 17 files changed, 505 insertions(+), 88 deletions(-) create mode 100644 src/Core/PostgreSQL/Connection.cpp create mode 100644 src/Core/PostgreSQL/Connection.h create mode 100644 src/Core/PostgreSQL/ConnectionHolder.h create mode 100644 src/Core/PostgreSQL/PoolWithFailover.cpp create mode 100644 src/Core/PostgreSQL/PoolWithFailover.h create mode 100644 src/Core/PostgreSQL/Utils.cpp create mode 100644 src/Core/PostgreSQL/Utils.h create mode 100644 tests/integration/test_dictionaries_update_field/__init__.py create mode 100644 tests/integration/test_dictionaries_update_field/configs/config.xml create mode 100644 tests/integration/test_dictionaries_update_field/configs/users.xml diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp new file mode 100644 index 00000000000..ff6197d1390 --- /dev/null +++ b/src/Core/PostgreSQL/Connection.cpp @@ -0,0 +1,35 @@ +#include "Connection.h" +#include + +namespace postgres +{ + +Connection::Connection(const ConnectionInfo & connection_info_, bool replication_) + : connection_info(connection_info_), replication(replication_) +{ + if (replication) + { + connection_info = std::make_pair( + fmt::format("{} replication=database", connection_info.first), connection_info.second); + } +} + +pqxx::connection & Connection::getRef() +{ + connect(); + assert(connection != nullptr); + return *connection; +} + +void Connection::connect() +{ + if (!connection || !connection->is_open()) + { + /// Always throws if there is no connection. + connection = std::make_unique(connection_info.first); + if (replication) + connection->set_variable("default_transaction_isolation", "'repeatable read'"); + LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second); + } +} +} diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h new file mode 100644 index 00000000000..1e9334eace5 --- /dev/null +++ b/src/Core/PostgreSQL/Connection.h @@ -0,0 +1,30 @@ +#pragma once + +#include // Y_IGNORE +#include + + +namespace postgres +{ +using ConnectionInfo = std::pair; +using ConnectionPtr = std::unique_ptr; + +class Connection +{ +public: + Connection(const ConnectionInfo & connection_info_, bool replication_ = false); + + Connection(const Connection & other) = delete; + + pqxx::connection & getRef(); + + void connect(); + + const ConnectionInfo & getConnectionInfo() { return connection_info; } + +private: + ConnectionPtr connection; + ConnectionInfo connection_info; + bool replication; +}; +} diff --git a/src/Core/PostgreSQL/ConnectionHolder.h b/src/Core/PostgreSQL/ConnectionHolder.h new file mode 100644 index 00000000000..98ab7df182d --- /dev/null +++ b/src/Core/PostgreSQL/ConnectionHolder.h @@ -0,0 +1,37 @@ +#pragma once + +#include // Y_IGNORE +#include +#include + + +namespace postgres +{ + +using ConnectionPtr = std::unique_ptr; +using Pool = BorrowedObjectPool; +using PoolPtr = std::shared_ptr; + +class ConnectionHolder +{ + +public: + ConnectionHolder(PoolPtr pool_, ConnectionPtr connection_) : pool(pool_), connection(std::move(connection_)) {} + + ConnectionHolder(const ConnectionHolder & other) = delete; + + ~ConnectionHolder() { pool->returnObject(std::move(connection)); } + + pqxx::connection & get() + { + assert(connection != nullptr); + return *connection; + } + +private: + PoolPtr pool; + ConnectionPtr connection; +}; + +using ConnectionHolderPtr = std::unique_ptr; +} diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp new file mode 100644 index 00000000000..6bf756b8a12 --- /dev/null +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -0,0 +1,138 @@ +#include "PoolWithFailover.h" +#include "Utils.h" +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int POSTGRESQL_CONNECTION_FAILURE; +} +} + +namespace postgres +{ + +PoolWithFailover::PoolWithFailover( + const Poco::Util::AbstractConfiguration & config, const String & config_prefix, + size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_) + : pool_wait_timeout(pool_wait_timeout_) + , max_tries(max_tries_) +{ + LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", + pool_size, pool_wait_timeout, max_tries_); + + auto db = config.getString(config_prefix + ".db", ""); + auto host = config.getString(config_prefix + ".host", ""); + auto port = config.getUInt(config_prefix + ".port", 0); + auto user = config.getString(config_prefix + ".user", ""); + auto password = config.getString(config_prefix + ".password", ""); + + if (config.has(config_prefix + ".replica")) + { + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_prefix, config_keys); + + for (const auto & config_key : config_keys) + { + if (config_key.starts_with("replica")) + { + std::string replica_name = config_prefix + "." + config_key; + size_t priority = config.getInt(replica_name + ".priority", 0); + + auto replica_host = config.getString(replica_name + ".host", host); + auto replica_port = config.getUInt(replica_name + ".port", port); + auto replica_user = config.getString(replica_name + ".user", user); + auto replica_password = config.getString(replica_name + ".password", password); + + auto connection_string = formatConnectionString(db, replica_host, replica_port, replica_user, replica_password).first; + replicas_with_priority[priority].emplace_back(connection_string, pool_size); + } + } + } + else + { + auto connection_string = formatConnectionString(db, host, port, user, password).first; + replicas_with_priority[0].emplace_back(connection_string, pool_size); + } +} + +PoolWithFailover::PoolWithFailover( + const std::string & database, + const RemoteDescription & addresses, + const std::string & user, const std::string & password, + size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_) + : pool_wait_timeout(pool_wait_timeout_) + , max_tries(max_tries_) +{ + LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", + pool_size, pool_wait_timeout, max_tries_); + + /// Replicas have the same priority, but traversed replicas are moved to the end of the queue. + for (const auto & [host, port] : addresses) + { + LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port); + auto connection_string = formatConnectionString(database, host, port, user, password).first; + replicas_with_priority[0].emplace_back(connection_string, pool_size); + } +} + +ConnectionHolderPtr PoolWithFailover::get() +{ + std::lock_guard lock(mutex); + + for (size_t try_idx = 0; try_idx < max_tries; ++try_idx) + { + for (auto & priority : replicas_with_priority) + { + auto & replicas = priority.second; + for (size_t i = 0; i < replicas.size(); ++i) + { + auto & replica = replicas[i]; + + ConnectionPtr connection; + auto connection_available = replica.pool->tryBorrowObject(connection, []() { return nullptr; }, pool_wait_timeout); + + if (!connection_available) + { + LOG_WARNING(log, "Unable to fetch connection within the timeout"); + continue; + } + + try + { + /// Create a new connection or reopen an old connection if it became invalid. + if (!connection || !connection->is_open()) + { + connection = std::make_unique(replica.connection_string); + LOG_DEBUG(log, "New connection to {}:{}", connection->hostname(), connection->port()); + } + } + catch (const pqxx::broken_connection & pqxx_error) + { + LOG_ERROR(log, "Connection error: {}", pqxx_error.what()); + + replica.pool->returnObject(std::move(connection)); + continue; + } + catch (...) + { + replica.pool->returnObject(std::move(connection)); + throw; + } + + auto connection_holder = std::make_unique(replica.pool, std::move(connection)); + + /// Move all traversed replicas to the end. + if (replicas.size() > 1) + std::rotate(replicas.begin(), replicas.begin() + i + 1, replicas.end()); + + return connection_holder; + } + } + } + + throw DB::Exception(DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, "Unable to connect to any of the replicas"); +} +} diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h new file mode 100644 index 00000000000..f4ae2c6cd1b --- /dev/null +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -0,0 +1,65 @@ +#pragma once + +#include "ConnectionHolder.h" +#include +#include +#include + + +namespace postgres +{ + +class PoolWithFailover +{ + +using RemoteDescription = std::vector>; + +public: + static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16; + static constexpr inline auto POSTGRESQL_POOL_WAIT_TIMEOUT = 5000; + static constexpr inline auto POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES = 5; + + PoolWithFailover( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE, + size_t pool_wait_timeout = POSTGRESQL_POOL_WAIT_TIMEOUT, + size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); + + PoolWithFailover( + const std::string & database, + const RemoteDescription & addresses, + const std::string & user, + const std::string & password, + size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE, + size_t pool_wait_timeout = POSTGRESQL_POOL_WAIT_TIMEOUT, + size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); + + PoolWithFailover(const PoolWithFailover & other) = delete; + + ConnectionHolderPtr get(); + +private: + struct PoolHolder + { + String connection_string; + PoolPtr pool; + + PoolHolder(const String & connection_string_, size_t pool_size) + : connection_string(connection_string_), pool(std::make_shared(pool_size)) {} + }; + + /// Highest priority is 0, the bigger the number in map, the less the priority + using Replicas = std::vector; + using ReplicasWithPriority = std::map; + + ReplicasWithPriority replicas_with_priority; + size_t pool_wait_timeout; + size_t max_tries; + std::mutex mutex; + Poco::Logger * log = &Poco::Logger::get("PostgreSQLConnectionPool"); +}; + +using PoolWithFailoverPtr = std::shared_ptr; + +} diff --git a/src/Core/PostgreSQL/Utils.cpp b/src/Core/PostgreSQL/Utils.cpp new file mode 100644 index 00000000000..98e76da99d2 --- /dev/null +++ b/src/Core/PostgreSQL/Utils.cpp @@ -0,0 +1,19 @@ +#include "Utils.h" +#include + +namespace postgres +{ + +ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password) +{ + DB::WriteBufferFromOwnString out; + out << "dbname=" << DB::quote << dbname + << " host=" << DB::quote << host + << " port=" << port + << " user=" << DB::quote << user + << " password=" << DB::quote << password + << " connect_timeout=10"; + return std::make_pair(out.str(), host + ':' + DB::toString(port)); +} + +} diff --git a/src/Core/PostgreSQL/Utils.h b/src/Core/PostgreSQL/Utils.h new file mode 100644 index 00000000000..ccb133112d9 --- /dev/null +++ b/src/Core/PostgreSQL/Utils.h @@ -0,0 +1,37 @@ +#pragma once + +#include // Y_IGNORE +#include +#include "Connection.h" + +namespace pqxx +{ + using ReadTransaction = pqxx::read_transaction; + using ReplicationTransaction = pqxx::transaction; +} + + +namespace postgres +{ + +ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password); + +Connection createReplicationConnection(const ConnectionInfo & connection_info); + +template +class Transaction +{ +public: + Transaction(pqxx::connection & connection) : transaction(connection) {} + + ~Transaction() { transaction.commit(); } + + T & getRef() { return transaction; } + + void exec(const String & query) { transaction.exec(query); } + +private: + T transaction; +}; + +} diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 7eabce7c270..79a98b7b070 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( ContextPtr context_, - postgres::Connection && connection_, + std::shared_ptr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, @@ -37,7 +37,7 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( , replication_slot_name(replication_slot_name_) , publication_name(publication_name_) , metadata(metadata_path) - , connection(std::move(connection_)) + , connection(connection_) , current_lsn(start_lsn) , max_block_size(max_block_size_) , allow_automatic_update(allow_automatic_update_) @@ -88,7 +88,7 @@ void MaterializePostgreSQLConsumer::readMetadata() if (!metadata.lsn().empty()) { - auto tx = std::make_shared(connection.getRef()); + auto tx = std::make_shared(connection->getRef()); final_lsn = metadata.lsn(); final_lsn = advanceLSN(tx); tx->commit(); @@ -600,7 +600,7 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() try { - tx = std::make_shared(connection.getRef()); + tx = std::make_shared(connection->getRef()); /// Read up to max_block_size rows changes (upto_n_changes parameter). It might return larger number as the limit /// is checked only after each transaction block. diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index ffe80c93ca6..afb39519715 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -28,7 +28,7 @@ public: MaterializePostgreSQLConsumer( ContextPtr context_, - postgres::Connection && connection_, + std::shared_ptr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, const std::string & metadata_path, @@ -106,7 +106,7 @@ private: const std::string replication_slot_name, publication_name; MaterializePostgreSQLMetadata metadata; - postgres::Connection connection; + std::shared_ptr connection; std::string current_lsn, final_lsn; const size_t max_block_size; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index dc38d18759e..30d3f1e6e97 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB @@ -42,7 +42,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , allow_automatic_update(allow_automatic_update_) , is_materialize_postgresql_database(is_materialize_postgresql_database_) , tables_list(tables_list_) - , connection(connection_info_) + , connection(std::make_shared(connection_info_)) { replication_slot = fmt::format("{}_ch_replication_slot", current_database_name); publication_name = fmt::format("{}_ch_publication", current_database_name); @@ -68,8 +68,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() { try { - /// Will throw pqxx::broken_connection if no connection at the moment - connection.isValid(); + connection->connect(); /// Will throw pqxx::broken_connection if no connection at the moment startSynchronization(false); } catch (const pqxx::broken_connection & pqxx_error) @@ -95,7 +94,7 @@ void PostgreSQLReplicationHandler::shutdown() void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { { - postgres::Transaction tx(connection.getRef()); + postgres::Transaction tx(connection->getRef()); createPublicationIfNeeded(tx.getRef()); } @@ -121,6 +120,8 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); + /// Throw in case of single MaterializePostgreSQL storage, becuase initial setup is done immediately + /// (unlike database engine where it is done in a separate thread). if (throw_on_error) throw; } @@ -134,16 +135,17 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { initial_sync(); } + /// Replication slot depends on publication, so if replication slot exists and new + /// publication was just created - drop that replication slot and start from scratch. else if (new_publication_created) { - /// Replication slot depends on publication, so if replication slot exists and new - /// publication was just created - drop that replication slot and start from scratch. dropReplicationSlot(tx.getRef()); initial_sync(); } + /// Synchronization and initial load already took place - do not create any new tables, just fetch StoragePtr's + /// and pass them to replication consumer. else { - /// Synchronization and initial load already took place. LOG_TRACE(log, "Loading {} tables...", materialized_storages.size()); for (const auto & [table_name, storage] : materialized_storages) { @@ -179,9 +181,12 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } } + /// Pass current connection to consumer. It is not std::moved implicitly, but a shared_ptr is passed. + /// Consumer and replication handler are always executed one after another (not concurrently) and share the same connection. + /// Handler uses it only for loadFromSnapshot and shutdown methods. consumer = std::make_shared( context, - std::move(connection), + connection, replication_slot, publication_name, metadata_path, @@ -197,10 +202,10 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } -StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_name, const String & table_name, +StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name, const String & table_name, StorageMaterializePostgreSQL * materialized_storage) { - auto tx = std::make_shared(connection.getRef()); + auto tx = std::make_shared(connection->getRef()); std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); @@ -242,7 +247,16 @@ void PostgreSQLReplicationHandler::consumerFunc() bool schedule_now = consumer->consume(skipped_tables); if (!skipped_tables.empty()) - reloadFromSnapshot(skipped_tables); + { + try + { + reloadFromSnapshot(skipped_tables); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } if (stop_synchronization) return; @@ -270,6 +284,7 @@ bool PostgreSQLReplicationHandler::isPublicationExist(pqxx::work & tx) void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bool create_without_check) { + /// For database engine a publication can be created earlier than in startReplication(). if (new_publication_created) return; @@ -370,12 +385,10 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::shutdownFinal() { - if (Poco::File(metadata_path).exists()) - Poco::File(metadata_path).remove(); - - postgres::Connection connection_(connection_info); - postgres::Transaction tx(connection_.getRef()); + if (std::filesystem::exists(metadata_path)) + std::filesystem::remove(metadata_path); + postgres::Transaction tx(connection->getRef()); dropPublication(tx.getRef()); if (isReplicationSlotExist(tx.getRef(), replication_slot)) dropReplicationSlot(tx.getRef()); @@ -432,80 +445,69 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector tx(replication_connection.getRef()); + + std::string snapshot_name, start_lsn; + createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, true); + + for (const auto & [relation_id, table_name] : relation_data) { - postgres::Connection replication_connection(connection_info, /* replication */true); - postgres::Transaction tx(replication_connection.getRef()); + auto storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context); + auto * materialized_storage = storage->as (); - std::string snapshot_name, start_lsn; - createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, true); + auto temp_materialized_storage = materialized_storage->createTemporary(); - for (const auto & [relation_id, table_name] : relation_data) + /// This snapshot is valid up to the end of the transaction, which exported it. + StoragePtr temp_nested_storage = loadFromSnapshot(snapshot_name, table_name, temp_materialized_storage->as ()); + + auto table_id = materialized_storage->getNestedStorageID(); + auto temp_table_id = temp_nested_storage->getStorageID(); + + LOG_TRACE(log, "Starting background update of table {}.{} ({}) with table {}.{} ({})", + table_id.database_name, table_id.table_name, toString(table_id.uuid), + temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); + + auto ast_rename = std::make_shared(); + ASTRenameQuery::Element elem { - auto storage = DatabaseCatalog::instance().getTable( - StorageID(current_database_name, table_name), - context); - auto * materialized_storage = storage->as (); + ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, + ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} + }; + ast_rename->elements.push_back(std::move(elem)); + ast_rename->exchange = true; - auto temp_materialized_storage = materialized_storage->createTemporary(); + auto nested_context = materialized_storage->getNestedTableContext(); - /// This snapshot is valid up to the end of the transaction, which exported it. - StoragePtr temp_nested_storage = loadFromSnapshot(snapshot_name, table_name, temp_materialized_storage->as ()); + try + { + auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + InterpreterRenameQuery(ast_rename, nested_context).execute(); - auto table_id = materialized_storage->getNestedStorageID(); - auto temp_table_id = temp_nested_storage->getStorageID(); - - LOG_TRACE(log, "Starting background update of table {}.{} ({}) with table {}.{} ({})", - table_id.database_name, table_id.table_name, toString(table_id.uuid), - temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); - - auto ast_rename = std::make_shared(); - ASTRenameQuery::Element elem { - ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, - ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} - }; - ast_rename->elements.push_back(std::move(elem)); - ast_rename->exchange = true; + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); + auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + auto nested_table_id = nested_storage->getStorageID(); - auto nested_context = materialized_storage->getNestedTableContext(); + materialized_storage->setNestedStorageID(nested_table_id); + nested_storage = materialized_storage->prepare(); + LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); - try - { - auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); - InterpreterRenameQuery(ast_rename, nested_context).execute(); - - { - auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); - auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); - auto nested_table_id = nested_storage->getStorageID(); - - materialized_storage->setNestedStorageID(nested_table_id); - nested_storage = materialized_storage->prepare(); - LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); - - /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. - consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); - } - - LOG_DEBUG(log, "Dropping table {}.{} ({})", temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true); - - dropReplicationSlot(tx.getRef(), /* temporary */true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); + /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. + consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); } + + LOG_DEBUG(log, "Dropping table {}.{} ({})", temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + + dropReplicationSlot(tx.getRef(), /* temporary */true); } - - } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index ddea1d03763..c955b2fbe3a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -113,7 +113,8 @@ private: String replication_slot, publication_name; - postgres::Connection connection; + /// Shared between replication_consumer and replication_handler, but never accessed concurrently. + std::shared_ptr connection; /// Replication consumer. Manages decoding of replication stream and syncing into tables. std::shared_ptr consumer; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h index 6b896c24dfa..e38041a1b78 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h @@ -173,7 +173,7 @@ private: /// It results in the fact: single MaterializePostgreSQL storage is created only if its nested table is created. /// In case of attach - this setup will be done in a separate thread in the background. It will also /// be checked for nested table and attempted to load it if it does not exist for some reason. - bool is_attach; + bool is_attach = true; }; } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index db609cd6081..54facb9ca0b 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -43,9 +43,9 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr context) const { const bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls; - auto connection = connection_pool->get(); + auto connection_holder = connection_pool->get(); auto columns = fetchPostgreSQLTableStructure( - connection->conn(), + connection_holder->get(), remote_table_schema.empty() ? doubleQuoteString(remote_table_name) : doubleQuoteString(remote_table_schema) + '.' + doubleQuoteString(remote_table_name), use_nulls).columns; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index a3d024c1a50..c31d02fa955 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -5,7 +5,7 @@ #if USE_LIBPQXX #include -#include +#include namespace DB diff --git a/tests/integration/test_dictionaries_update_field/__init__.py b/tests/integration/test_dictionaries_update_field/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_dictionaries_update_field/configs/config.xml b/tests/integration/test_dictionaries_update_field/configs/config.xml new file mode 100644 index 00000000000..a1518083be3 --- /dev/null +++ b/tests/integration/test_dictionaries_update_field/configs/config.xml @@ -0,0 +1,30 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_update_field/configs/users.xml b/tests/integration/test_dictionaries_update_field/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/tests/integration/test_dictionaries_update_field/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + From 4ac023e511ad5b3afbb16c1286e37ac131ea7ce8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 9 May 2021 22:18:16 +0000 Subject: [PATCH 081/105] Small fixes --- src/Core/PostgreSQL/Connection.h | 5 ++--- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 3 ++- src/Core/ya.make.in | 2 +- .../PostgreSQL/DatabaseMaterializePostgreSQL.cpp | 16 ++++++++-------- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 6 +++--- .../PostgreSQL/MaterializePostgreSQLConsumer.cpp | 6 +++--- .../PostgreSQL/MaterializePostgreSQLMetadata.cpp | 1 - .../PostgreSQL/MaterializePostgreSQLMetadata.h | 2 +- 8 files changed, 20 insertions(+), 21 deletions(-) diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h index 1e9334eace5..46646ea6f35 100644 --- a/src/Core/PostgreSQL/Connection.h +++ b/src/Core/PostgreSQL/Connection.h @@ -2,6 +2,7 @@ #include // Y_IGNORE #include +#include namespace postgres @@ -9,13 +10,11 @@ namespace postgres using ConnectionInfo = std::pair; using ConnectionPtr = std::unique_ptr; -class Connection +class Connection : private boost::noncopyable { public: Connection(const ConnectionInfo & connection_info_, bool replication_ = false); - Connection(const Connection & other) = delete; - pqxx::connection & getRef(); void connect(); diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 70537767dc5..07f2404fdc3 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -131,7 +131,8 @@ void insertPostgreSQLValue( { max_dimension = std::max(max_dimension, dimension); - if (--dimension == 0) + --dimension; + if (dimension == 0) break; dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); diff --git a/src/Core/ya.make.in b/src/Core/ya.make.in index f17aef89c54..6b89dc18482 100644 --- a/src/Core/ya.make.in +++ b/src/Core/ya.make.in @@ -10,7 +10,7 @@ PEERDIR( SRCS( - + ) END() diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 163e38f79f2..af6e2c1edfd 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -42,7 +42,7 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const String & postgres_database_name, const postgres::ConnectionInfo & connection_info_, std::unique_ptr settings_) - : DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context_) + : DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context_) , database_engine_define(database_engine_define_->clone()) , remote_database_name(postgres_database_name) , connection_info(connection_info_) @@ -72,16 +72,16 @@ void DatabaseMaterializePostgreSQL::startSynchronization() /// Check nested ReplacingMergeTree table. auto storage = DatabaseAtomic::tryGetTable(table_name, getContext()); - if (!storage) - { - /// Nested table does not exist and will be created by replication thread. - storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), getContext()); - } - else + if (storage) { /// Nested table was already created and synchronized. storage = StorageMaterializePostgreSQL::create(storage, getContext()); } + else + { + /// Nested table does not exist and will be created by replication thread. + storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), getContext()); + } /// Cache MaterializePostgreSQL wrapper over nested table. materialized_tables[table_name] = storage; @@ -90,7 +90,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() replication_handler->addStorage(table_name, storage->as()); } - LOG_TRACE(log, "Loaded {} tables. Starting synchronization, (database: {})", materialized_tables.size(), database_name); + LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); replication_handler->startup(); } diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 8b933f1e6a5..199b66b7160 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -179,7 +179,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "FROM pg_index i " "JOIN pg_attribute a ON a.attrelid = i.indrelid " "AND a.attnum = ANY(i.indkey) " - "WHERE i.indrelid = '{}'::regclass AND i.indisprimary", postgres_table_name); + "WHERE i.indrelid = {}::regclass AND i.indisprimary", quoteString(postgres_table_name)); table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); } @@ -201,10 +201,10 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "and a.attrelid = t.oid " "and a.attnum = ANY(ix.indkey) " "and t.relkind = 'r' " /// simple tables - "and t.relname = '{}' " /// Connection is already done to a needed database, only table name is needed. + "and t.relname = {} " /// Connection is already done to a needed database, only table name is needed. "and ix.indisreplident = 't' " /// index is is replica identity index "ORDER BY a.attname", /// column names - postgres_table_name); + quoteString(postgres_table_name)); table.replica_identity_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true); } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 79a98b7b070..43f6f48a4ef 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -241,14 +241,14 @@ void MaterializePostgreSQLConsumer::readTupleData( case PostgreSQLQuery::INSERT: { buffer.columns[num_columns]->insert(Int8(1)); - buffer.columns[num_columns + 1]->insert(UInt64(metadata.version())); + buffer.columns[num_columns + 1]->insert(UInt64(metadata.getAndIncrementVersion())); break; } case PostgreSQLQuery::DELETE: { buffer.columns[num_columns]->insert(Int8(-1)); - buffer.columns[num_columns + 1]->insert(UInt64(metadata.version())); + buffer.columns[num_columns + 1]->insert(UInt64(metadata.getAndIncrementVersion())); break; } @@ -260,7 +260,7 @@ void MaterializePostgreSQLConsumer::readTupleData( else buffer.columns[num_columns]->insert(Int8(1)); - buffer.columns[num_columns + 1]->insert(UInt64(metadata.version())); + buffer.columns[num_columns + 1]->insert(UInt64(metadata.getAndIncrementVersion())); break; } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp index ee945c67c1a..66fee26a5e8 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp @@ -78,7 +78,6 @@ void MaterializePostgreSQLMetadata::commitMetadata(std::string & lsn, const std: try { actual_lsn = finalizeStreamFunc(); - /// This is not supposed to happen if (actual_lsn != last_lsn) { diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h index d09adb61363..8895707450b 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h @@ -14,7 +14,7 @@ public: void readMetadata(); - size_t version() { return last_version++; } + size_t getAndIncrementVersion() { return last_version++; } std::string lsn() { return last_lsn; } From ae1191d0c069d1a1d37990a905219437b4e4755f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 May 2021 09:10:02 +0000 Subject: [PATCH 082/105] Get rid of metadata file, rely only on lsn --- .../DatabaseMaterializePostgreSQL.cpp | 9 -- .../MaterializePostgreSQLConsumer.cpp | 71 ++++--------- .../MaterializePostgreSQLConsumer.h | 26 ++--- .../MaterializePostgreSQLMetadata.cpp | 100 ------------------ .../MaterializePostgreSQLMetadata.h | 31 ------ .../PostgreSQLReplicationHandler.cpp | 40 +++---- .../PostgreSQL/PostgreSQLReplicationHandler.h | 19 +--- .../StorageMaterializePostgreSQL.cpp | 5 - 8 files changed, 54 insertions(+), 247 deletions(-) delete mode 100644 src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp delete mode 100644 src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index af6e2c1edfd..dda21937fa2 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -31,8 +31,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static const auto METADATA_SUFFIX = ".materialize_postgresql_metadata"; - DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( ContextPtr context_, const String & metadata_path_, @@ -57,7 +55,6 @@ void DatabaseMaterializePostgreSQL::startSynchronization() remote_database_name, database_name, connection_info, - metadata_path + METADATA_SUFFIX, getContext(), settings->materialize_postgresql_max_block_size.value, settings->materialize_postgresql_allow_automatic_update, @@ -189,12 +186,6 @@ void DatabaseMaterializePostgreSQL::drop(ContextPtr local_context) if (replication_handler) replication_handler->shutdownFinal(); - /// Remove metadata - Poco::File metadata(getMetadataPath() + METADATA_SUFFIX); - - if (metadata.exists()) - metadata.remove(false); - DatabaseAtomic::drop(StorageMaterializePostgreSQL::makeNestedTableContext(local_context)); } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 43f6f48a4ef..1a6e635f336 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -1,8 +1,6 @@ #include "MaterializePostgreSQLConsumer.h" -#if USE_LIBPQXX #include "StorageMaterializePostgreSQL.h" - #include #include #include @@ -27,7 +25,6 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( std::shared_ptr connection_, const std::string & replication_slot_name_, const std::string & publication_name_, - const std::string & metadata_path, const std::string & start_lsn, const size_t max_block_size_, bool allow_automatic_update_, @@ -36,7 +33,6 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( , context(context_) , replication_slot_name(replication_slot_name_) , publication_name(publication_name_) - , metadata(metadata_path) , connection(connection_) , current_lsn(start_lsn) , max_block_size(max_block_size_) @@ -80,27 +76,6 @@ void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage } -void MaterializePostgreSQLConsumer::readMetadata() -{ - try - { - metadata.readMetadata(); - - if (!metadata.lsn().empty()) - { - auto tx = std::make_shared(connection->getRef()); - final_lsn = metadata.lsn(); - final_lsn = advanceLSN(tx); - tx->commit(); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - void MaterializePostgreSQLConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); @@ -241,14 +216,14 @@ void MaterializePostgreSQLConsumer::readTupleData( case PostgreSQLQuery::INSERT: { buffer.columns[num_columns]->insert(Int8(1)); - buffer.columns[num_columns + 1]->insert(UInt64(metadata.getAndIncrementVersion())); + buffer.columns[num_columns + 1]->insert(lsn_value); break; } case PostgreSQLQuery::DELETE: { buffer.columns[num_columns]->insert(Int8(-1)); - buffer.columns[num_columns + 1]->insert(UInt64(metadata.getAndIncrementVersion())); + buffer.columns[num_columns + 1]->insert(lsn_value); break; } @@ -260,7 +235,7 @@ void MaterializePostgreSQLConsumer::readTupleData( else buffer.columns[num_columns]->insert(Int8(1)); - buffer.columns[num_columns + 1]->insert(UInt64(metadata.getAndIncrementVersion())); + buffer.columns[num_columns + 1]->insert(lsn_value); break; } @@ -488,30 +463,27 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptr(); - insert->table_id = storage->getStorageID(); - insert->columns = buffer.columnsAST; + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + insert->columns = buffer.columnsAST; - auto insert_context = Context::createCopy(context); - insert_context->makeQueryContext(); - insert_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + auto insert_context = Context::createCopy(context); + insert_context->makeQueryContext(); + insert_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - InterpreterInsertQuery interpreter(insert, insert_context, true); - auto block_io = interpreter.execute(); - OneBlockInputStream input(result_rows); + InterpreterInsertQuery interpreter(insert, insert_context, true); + auto block_io = interpreter.execute(); + OneBlockInputStream input(result_rows); - assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica table sync"); - copyData(input, *block_io.out); + assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica table sync"); + copyData(input, *block_io.out); - auto actual_lsn = advanceLSN(tx); - buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); - - return actual_lsn; - }); + /// The next attempt to read data will start with actual_lsn, returned from advanceLSN. current_lsn acts as + /// a version for rows in RelplacingMergeTree table. + current_lsn = advanceLSN(tx); + buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); } } catch (...) @@ -632,6 +604,8 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() slot_empty = false; current_lsn = (*row)[0]; + lsn_value = getLSNValue(current_lsn); + LOG_DEBUG(log, "Current lsn: {}, value: {}", current_lsn, lsn_value); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } @@ -704,7 +678,4 @@ void MaterializePostgreSQLConsumer::updateNested(const String & table_name, Stor skip_list[table_id] = table_start_lsn; } - } - -#endif diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index afb39519715..43ac919f119 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -1,11 +1,5 @@ #pragma once -#if !defined(ARCADIA_BUILD) -#include "config_core.h" -#endif - -#if USE_LIBPQXX -#include "MaterializePostgreSQLMetadata.h" #include #include @@ -15,7 +9,6 @@ #include #include #include -#include "pqxx/pqxx" // Y_IGNORE namespace DB @@ -29,16 +22,13 @@ public: MaterializePostgreSQLConsumer( ContextPtr context_, std::shared_ptr connection_, - const std::string & replication_slot_name_, - const std::string & publication_name_, - const std::string & metadata_path, - const std::string & start_lsn, + const String & replication_slot_name_, + const String & publication_name_, + const String & start_lsn, const size_t max_block_size_, bool allow_automatic_update_, Storages storages_); - void readMetadata(); - bool consume(std::vector> & skipped_tables); /// Called from reloadFromSnapshot by replication handler. This method is needed to move a table back into synchronization @@ -105,14 +95,17 @@ private: ContextPtr context; const std::string replication_slot_name, publication_name; - MaterializePostgreSQLMetadata metadata; std::shared_ptr connection; std::string current_lsn, final_lsn; + + /// current_lsn converted from String to Int64 via getLSNValue(). + UInt64 lsn_value; + const size_t max_block_size; bool allow_automatic_update; - std::string table_to_insert; + String table_to_insert; /// List of tables which need to be synced after last replication stream. std::unordered_set tables_to_sync; @@ -147,7 +140,4 @@ private: /// i.e. we will not miss the first start_lsn position for reloaded table. std::unordered_map skip_list; }; - } - -#endif diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp deleted file mode 100644 index 66fee26a5e8..00000000000 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.cpp +++ /dev/null @@ -1,100 +0,0 @@ -#include "MaterializePostgreSQLMetadata.h" - -#if USE_LIBPQXX -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -MaterializePostgreSQLMetadata::MaterializePostgreSQLMetadata(const std::string & metadata_file_path) - : metadata_file(metadata_file_path) - , tmp_metadata_file(metadata_file_path + ".tmp") - , last_version(1) -{ -} - - -void MaterializePostgreSQLMetadata::readMetadata() -{ - if (Poco::File(metadata_file).exists()) - { - ReadBufferFromFile in(metadata_file, DBMS_DEFAULT_BUFFER_SIZE); - - assertString("\nLast version:\t", in); - readIntText(last_version, in); - - assertString("\nLast LSN:\t", in); - readString(last_lsn, in); - - if (checkString("\nActual LSN:\t", in)) - { - std::string actual_lsn; - readString(actual_lsn, in); - - if (!actual_lsn.empty()) - last_lsn = actual_lsn; - } - - LOG_DEBUG(&Poco::Logger::get("MaterializePostgreSQLMetadata"), - "Last written version is {}. (From metadata file {})", last_version, metadata_file); - } -} - - -void MaterializePostgreSQLMetadata::writeMetadata(bool append_metadata) -{ - WriteBufferFromFile out(tmp_metadata_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); - - if (append_metadata) - { - writeString("\nActual LSN:\t" + toString(last_lsn), out); - } - else - { - writeString("\nLast version:\t" + toString(last_version), out); - writeString("\nLast LSN:\t" + toString(last_lsn), out); - } - - out.next(); - out.sync(); - out.close(); -} - - -/// While data is received, version is updated. Before table sync, write last version to tmp file. -/// Then sync data to table and rename tmp to non-tmp. -void MaterializePostgreSQLMetadata::commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc) -{ - std::string actual_lsn; - last_lsn = lsn; - writeMetadata(); - - try - { - actual_lsn = finalizeStreamFunc(); - /// This is not supposed to happen - if (actual_lsn != last_lsn) - { - writeMetadata(true); - LOG_WARNING(&Poco::Logger::get("MaterializePostgreSQLMetadata"), - "Last written LSN {} is not equal to actual LSN {}", last_lsn, actual_lsn); - } - - Poco::File(tmp_metadata_file).renameTo(metadata_file); - } - catch (...) - { - Poco::File(tmp_metadata_file).remove(); - throw; - } -} - -} - -#endif diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h b/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h deleted file mode 100644 index 8895707450b..00000000000 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLMetadata.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once -#include - - -namespace DB -{ - -class MaterializePostgreSQLMetadata -{ -public: - MaterializePostgreSQLMetadata(const std::string & metadata_file_path); - - void commitMetadata(std::string & lsn, const std::function & finalizeStreamFunc); - - void readMetadata(); - - size_t getAndIncrementVersion() { return last_version++; } - - std::string lsn() { return last_lsn; } - -private: - void writeMetadata(bool append_metadata = false); - - const std::string metadata_file; - const std::string tmp_metadata_file; - - uint64_t last_version; - std::string last_lsn; -}; - -} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 30d3f1e6e97..f9ed5eb9bbb 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,6 +1,5 @@ #include "PostgreSQLReplicationHandler.h" -#if USE_LIBPQXX #include #include #include @@ -8,8 +7,8 @@ #include #include #include +#include #include -#include namespace DB @@ -26,7 +25,6 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const String & remote_database_name_, const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, - const std::string & metadata_path_, ContextPtr context_, const size_t max_block_size_, bool allow_automatic_update_, @@ -36,7 +34,6 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , context(context_) , remote_database_name(remote_database_name_) , current_database_name(current_database_name_) - , metadata_path(metadata_path_) , connection_info(connection_info_) , max_block_size(max_block_size_) , allow_automatic_update(allow_automatic_update_) @@ -103,7 +100,15 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// List of nested tables (table_name -> nested_storage), which is passed to replication consumer. std::unordered_map nested_storages; - std::string snapshot_name, start_lsn; + + /// snapshot_name is initialized only if a new replication slot is created. + /// start_lsn is initialized in two places: + /// 1. if replication slot does not exist, start_lsn will be returned with its creation return parameters; + /// 2. if replication slot already exist, start_lsn is read from pg_replication_slots as + /// `confirmed_flush_lsn` - the address (LSN) up to which the logical slot's consumer has confirmed receiving data. + /// Data older than this is not available anymore. + /// TODO: more tests + String snapshot_name, start_lsn; auto initial_sync = [&]() { @@ -131,12 +136,13 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. /// Recreation of a replication slot imposes reloading of all tables. - if (!isReplicationSlotExist(tx.getRef(), replication_slot)) + if (!isReplicationSlotExist(tx.getRef(), replication_slot, start_lsn)) { initial_sync(); } /// Replication slot depends on publication, so if replication slot exists and new /// publication was just created - drop that replication slot and start from scratch. + /// TODO: tests else if (new_publication_created) { dropReplicationSlot(tx.getRef()); @@ -189,7 +195,6 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) connection, replication_slot, publication_name, - metadata_path, start_lsn, max_block_size, allow_automatic_update, @@ -317,24 +322,26 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bo } -bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & tx, std::string & slot_name) +bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & tx, String & slot_name, String & start_lsn) { - std::string query_str = fmt::format("SELECT active, restart_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); + std::string query_str = fmt::format("SELECT active, restart_lsn, confirmed_flush_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); pqxx::result result{tx.exec(query_str)}; /// Replication slot does not exist if (result.empty()) return false; - LOG_TRACE(log, "Replication slot {} already exists (active: {}). Restart lsn position is {}", - slot_name, result[0][0].as(), result[0][0].as()); + start_lsn = result[0][2].as(); + + LOG_TRACE(log, "Replication slot {} already exists (active: {}). Restart lsn position: {}, confirmed flush lsn: {}", + slot_name, result[0][0].as(), result[0][1].as(), start_lsn); return true; } void PostgreSQLReplicationHandler::createReplicationSlot( - pqxx::nontransaction & tx, std::string & start_lsn, std::string & snapshot_name, bool temporary) + pqxx::nontransaction & tx, String & start_lsn, String & snapshot_name, bool temporary) { std::string query_str; @@ -385,12 +392,10 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::shutdownFinal() { - if (std::filesystem::exists(metadata_path)) - std::filesystem::remove(metadata_path); - postgres::Transaction tx(connection->getRef()); dropPublication(tx.getRef()); - if (isReplicationSlotExist(tx.getRef(), replication_slot)) + String last_committed_lsn; + if (isReplicationSlotExist(tx.getRef(), replication_slot, last_committed_lsn)) dropReplicationSlot(tx.getRef()); } @@ -508,6 +513,5 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector #include @@ -29,7 +22,6 @@ public: const String & remote_database_name_, const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, - const String & metadata_path_, ContextPtr context_, const size_t max_block_size_, bool allow_automatic_update_, @@ -69,9 +61,9 @@ private: /// Methods to manage Replication Slots. - bool isReplicationSlotExist(pqxx::nontransaction & tx, std::string & slot_name); + bool isReplicationSlotExist(pqxx::nontransaction & tx, String & slot_name, String & start_lsn); - void createReplicationSlot(pqxx::nontransaction & tx, std::string & start_lsn, std::string & snapshot_name, bool temporary = false); + void createReplicationSlot(pqxx::nontransaction & tx, String & start_lsn, String & snapshot_name, bool temporary = false); void dropReplicationSlot(pqxx::nontransaction & tx, bool temporary = false); @@ -85,16 +77,13 @@ private: void reloadFromSnapshot(const std::vector> & relation_data); - PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const std::string & table_name); + PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name); Poco::Logger * log; ContextPtr context; const String remote_database_name, current_database_name; - /// Path for replication metadata. - const String metadata_path; - /// Connection string and address for logs. postgres::ConnectionInfo connection_info; @@ -133,5 +122,3 @@ private: }; } - -#endif diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 618f31c9c1b..4a7c9655149 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -63,15 +63,10 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( setInMemoryMetadata(storage_metadata); - /// Path to store replication metadata (like last written version, etc). - auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath() - + "/.metadata_" + table_id_.database_name + "_" + table_id_.table_name + "_" + toString(table_id_.uuid); - replication_handler = std::make_unique( remote_database_name, table_id_.database_name, connection_info, - metadata_path, getContext(), replication_settings->materialize_postgresql_max_block_size.value, /* allow_automatic_update */ false, /* is_materialize_postgresql_database */false); From 65c574db937936669aa1a600d58c915ff1876f03 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 May 2021 11:31:06 +0000 Subject: [PATCH 083/105] Add test --- .../MaterializePostgreSQLConsumer.cpp | 5 +- .../PostgreSQLReplicationHandler.cpp | 2 +- .../TableFunctionPostgreSQL.cpp | 1 - .../test.py | 76 +++++++++++++++++++ 4 files changed, 79 insertions(+), 5 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 1a6e635f336..5e41e20550e 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -35,6 +35,7 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( , publication_name(publication_name_) , connection(connection_) , current_lsn(start_lsn) + , lsn_value(getLSNValue(start_lsn)) , max_block_size(max_block_size_) , allow_automatic_update(allow_automatic_update_) , storages(storages_) @@ -249,6 +250,7 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli /// Skip '\x' size_t pos = 2; char type = readInt8(replication_message, pos, size); + //LOG_DEBUG(log, "Message type: {}, lsn string: {}, lsn value {}", type, current_lsn, lsn_value); switch (type) { @@ -480,8 +482,6 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptrgetHeader(), "postgresql replica table sync"); copyData(input, *block_io.out); - /// The next attempt to read data will start with actual_lsn, returned from advanceLSN. current_lsn acts as - /// a version for rows in RelplacingMergeTree table. current_lsn = advanceLSN(tx); buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); } @@ -605,7 +605,6 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() slot_empty = false; current_lsn = (*row)[0]; lsn_value = getLSNValue(current_lsn); - LOG_DEBUG(log, "Current lsn: {}, value: {}", current_lsn, lsn_value); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f9ed5eb9bbb..51b70bf1b5b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -125,7 +125,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); - /// Throw in case of single MaterializePostgreSQL storage, becuase initial setup is done immediately + /// Throw in case of single MaterializePostgreSQL storage, because initial setup is done immediately /// (unlike database engine where it is done in a separate thread). if (throw_on_error) throw; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 54facb9ca0b..de04f1fa7bd 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -14,7 +14,6 @@ #include "registerTableFunctions.h" #include #include -#include namespace DB diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index abfb3427c99..2bf3673c3a5 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -9,6 +9,9 @@ from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.test_tools import TSV +from random import randrange +import threading + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs = ['configs/log_conf.xml'], @@ -427,6 +430,79 @@ def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE test_database") +@pytest.mark.timeout(120) +def test_random_queries(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 5 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(10000)'.format(i)) + n = [10000] + + query = ['DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', + 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key > 6000;', + 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', + 'UPDATE postgresql_replica_{} SET value = value*value WHERE key < 5000;', + 'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;', + 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key < 5000;', + 'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;', + 'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 == 0;', + 'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;', + 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key >= 5000;', + 'DELETE FROM postgresql_replica_{} WHERE value-3 = 3;'] + + def attack(thread_id): + print('thread {}'.format(thread_id)) + k = 10000 + for i in range(10): + query_id = random.randrange(0, len(query)-1) + table_id = random.randrange(0, 5) # num tables + + # random update / delete query + cursor.execute(query[query_id].format(table_id)) + print("table {} query {} ok".format(table_id, query_id)) + + # allow some thread to do inserts (not to violate key constraints) + if thread_id < 5: + instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {} + number, number from numbers(1000)'.format(thread_id, k)) + k += 1 + print("insert table {} ok".format(thread_id)) + + threads = [] + threads_num = 16 + + for i in range(threads_num): + threads.append(threading.Thread(target=attack, args=(i,))) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + instance.query( + "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + + n[0] = 50000 + for table_id in range(NUM_TABLES): + n[0] += 1 + instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {} + number, number from numbers(5000)'.format(table_id, n[0])) + + for thread in threads: + thread.join() + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + count = instance.query('SELECT count() FROM test_database.postgresql_replica_{}'.format(i)) + print(count) + + for i in range(NUM_TABLES): + cursor.execute('drop table postgresql_replica_{};'.format(i)) + + instance.query("DROP DATABASE test_database") + assert 'test_database' not in instance.query('SHOW DATABASES') + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 626e87bae50c82380a19795fc18f2ad00d407b7a Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 May 2021 13:51:05 +0000 Subject: [PATCH 084/105] Some fixes --- src/Core/PostgreSQL/Utils.h | 13 ++++++++- .../DatabaseMaterializePostgreSQL.cpp | 7 ----- .../DatabaseMaterializePostgreSQL.h | 2 -- .../PostgreSQLReplicationHandler.cpp | 28 ++++++++++++------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- .../test.py | 22 ++++++++++----- 6 files changed, 46 insertions(+), 28 deletions(-) diff --git a/src/Core/PostgreSQL/Utils.h b/src/Core/PostgreSQL/Utils.h index ccb133112d9..b3c035ebac4 100644 --- a/src/Core/PostgreSQL/Utils.h +++ b/src/Core/PostgreSQL/Utils.h @@ -3,6 +3,7 @@ #include // Y_IGNORE #include #include "Connection.h" +#include namespace pqxx { @@ -24,7 +25,17 @@ class Transaction public: Transaction(pqxx::connection & connection) : transaction(connection) {} - ~Transaction() { transaction.commit(); } + ~Transaction() + { + try + { + transaction.commit(); + } + catch (...) + { + DB::tryLogCurrentException(__PRETTY_FUNCTION__); + } + } T & getRef() { return transaction; } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index dda21937fa2..080069c3f44 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -92,13 +92,6 @@ void DatabaseMaterializePostgreSQL::startSynchronization() } -void DatabaseMaterializePostgreSQL::shutdown() -{ - if (replication_handler) - replication_handler->shutdown(); -} - - void DatabaseMaterializePostgreSQL::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) { DatabaseAtomic::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h index 97399adf1ff..6a62491f940 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h @@ -55,8 +55,6 @@ public: void drop(ContextPtr local_context) override; - void shutdown() override; - void stopReplication(); private: diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 51b70bf1b5b..44eb48deec5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -136,7 +136,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. /// Recreation of a replication slot imposes reloading of all tables. - if (!isReplicationSlotExist(tx.getRef(), replication_slot, start_lsn)) + if (!isReplicationSlotExist(tx.getRef(), start_lsn, /* temporary */false)) { initial_sync(); } @@ -322,9 +322,15 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bo } -bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & tx, String & slot_name, String & start_lsn) +bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & tx, String & start_lsn, bool temporary) { - std::string query_str = fmt::format("SELECT active, restart_lsn, confirmed_flush_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); + String slot_name; + if (temporary) + slot_name = replication_slot + "_tmp"; + else + slot_name = replication_slot; + + String query_str = fmt::format("SELECT active, restart_lsn, confirmed_flush_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name); pqxx::result result{tx.exec(query_str)}; /// Replication slot does not exist @@ -343,9 +349,7 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & void PostgreSQLReplicationHandler::createReplicationSlot( pqxx::nontransaction & tx, String & start_lsn, String & snapshot_name, bool temporary) { - std::string query_str; - - std::string slot_name; + String query_str, slot_name; if (temporary) slot_name = replication_slot + "_tmp"; else @@ -395,8 +399,10 @@ void PostgreSQLReplicationHandler::shutdownFinal() postgres::Transaction tx(connection->getRef()); dropPublication(tx.getRef()); String last_committed_lsn; - if (isReplicationSlotExist(tx.getRef(), replication_slot, last_committed_lsn)) - dropReplicationSlot(tx.getRef()); + if (isReplicationSlotExist(tx.getRef(), last_committed_lsn, /* temporary */false)) + dropReplicationSlot(tx.getRef(), /* temporary */false); + if (isReplicationSlotExist(tx.getRef(), last_committed_lsn, /* temporary */true)) + dropReplicationSlot(tx.getRef(), /* temporary */true); } @@ -453,8 +459,10 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector tx(replication_connection.getRef()); - std::string snapshot_name, start_lsn; - createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, true); + String snapshot_name, start_lsn; + if (isReplicationSlotExist(tx.getRef(), start_lsn, /* temporary */true)) + dropReplicationSlot(tx.getRef(), /* temporary */true); + createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, /* temporary */true); for (const auto & [relation_id, table_name] : relation_data) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 7cf475603b1..61132d0c0fc 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -61,7 +61,7 @@ private: /// Methods to manage Replication Slots. - bool isReplicationSlotExist(pqxx::nontransaction & tx, String & slot_name, String & start_lsn); + bool isReplicationSlotExist(pqxx::nontransaction & tx, String & start_lsn, bool temporary = false); void createReplicationSlot(pqxx::nontransaction & tx, String & start_lsn, String & snapshot_name, bool temporary = false); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 2bf3673c3a5..e641523f9b1 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -443,16 +443,16 @@ def test_random_queries(started_cluster): n = [10000] query = ['DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', - 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key > 6000;', + 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;', 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', - 'UPDATE postgresql_replica_{} SET value = value*value WHERE key < 5000;', + 'UPDATE postgresql_replica_{} SET value = value*value WHERE key % 2 = 1;', 'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;', - 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key < 5000;', + 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;', 'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;', - 'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 == 0;', + 'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;', 'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;', - 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key >= 5000;', - 'DELETE FROM postgresql_replica_{} WHERE value-3 = 3;'] + 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;', + 'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;'] def attack(thread_id): print('thread {}'.format(thread_id)) @@ -467,10 +467,17 @@ def test_random_queries(started_cluster): # allow some thread to do inserts (not to violate key constraints) if thread_id < 5: - instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {} + number, number from numbers(1000)'.format(thread_id, k)) + print("try insert table {}".format(thread_id)) + instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {}*10000*({} + number), number from numbers(1000)'.format(i, thread_id, k)) k += 1 print("insert table {} ok".format(thread_id)) + if i == 5: + # also change primary key value + print("try update primary key {}".format(thread_id)) + cursor.execute("UPDATE postgresql_replica_{} SET key=key%100000+100000*{} WHERE key%{}=0".format(thread_id, i+1, i+1)) + print("update primary key {} ok".format(thread_id)) + threads = [] threads_num = 16 @@ -487,6 +494,7 @@ def test_random_queries(started_cluster): for table_id in range(NUM_TABLES): n[0] += 1 instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {} + number, number from numbers(5000)'.format(table_id, n[0])) + #cursor.execute("UPDATE postgresql_replica_{} SET key=key%100000+100000*{} WHERE key%{}=0".format(table_id, table_id+1, table_id+1)) for thread in threads: thread.join() From bd4a0934e4459158efdc29c9717a0a499c145bf5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 May 2021 14:51:17 +0000 Subject: [PATCH 085/105] Add one more test --- .../test.py | 56 +++++++++++++++++-- 1 file changed, 52 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index e641523f9b1..f16e0125efd 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -31,14 +31,15 @@ postgres_table_template_3 = """ key1 Integer NOT NULL, value1 Integer, key2 Integer NOT NULL, value2 Integer NOT NULL) """ -def get_postgres_conn(database=False): +def get_postgres_conn(database=False, auto_commit=True): if database == True: conn_string = "host='localhost' dbname='postgres_database' user='postgres' password='mysecretpassword'" else: conn_string = "host='localhost' user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True + if auto_commit: + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True return conn @@ -445,7 +446,7 @@ def test_random_queries(started_cluster): query = ['DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;', 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', - 'UPDATE postgresql_replica_{} SET value = value*value WHERE key % 2 = 1;', + 'UPDATE postgresql_replica_{} SET value = value*5 WHERE key % 2 = 1;', 'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;', 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;', 'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;', @@ -511,6 +512,53 @@ def test_random_queries(started_cluster): assert 'test_database' not in instance.query('SHOW DATABASES') +@pytest.mark.timeout(120) +def test_single_transaction(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(database=True, auto_commit=False) + cursor = conn.cursor() + + create_postgres_table(cursor, 'postgresql_replica_0'); + conn.commit() + instance.query( + "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + assert_nested_table_is_created('postgresql_replica_0') + + queries = [ + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(0, 10000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', + 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;', + "UPDATE postgresql_replica_{} SET key=key+20000 WHERE key%2=0", + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(40000, 50000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', + 'UPDATE postgresql_replica_{} SET value = value + 101 WHERE key % 2 = 1;', + "UPDATE postgresql_replica_{} SET key=key+80000 WHERE key%2=1", + 'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;', + 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;', + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(200000, 250000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;', + 'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;', + "UPDATE postgresql_replica_{} SET key=key+500000 WHERE key%2=1", + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(1000000, 1050000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;', + "UPDATE postgresql_replica_{} SET key=key+10000000", + 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;', + 'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;'] + + for query in queries: + print('query {}'.format(query)) + cursor.execute(query.format(0)) + + time.sleep(5) + result = instance.query("select count() from test_database.postgresql_replica_0") + # no commit yet + assert(int(result) == 0) + + conn.commit() + check_tables_are_synchronized('postgresql_replica_{}'.format(0)); + instance.query("DROP DATABASE test_database") + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 407db17e2ef16b1611e9bd22992c6c6cc5c4a00b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 12 May 2021 21:05:43 +0000 Subject: [PATCH 086/105] Remove setting, use native transaction --- src/Core/PostgreSQL/Utils.h | 31 ---------- src/Core/Settings.h | 1 - .../PostgreSQL/DatabasePostgreSQL.cpp | 3 +- .../fetchPostgreSQLTableStructure.cpp | 17 ++--- .../fetchPostgreSQLTableStructure.h | 4 +- .../PostgreSQLReplicationHandler.cpp | 62 +++++++++++-------- 6 files changed, 49 insertions(+), 69 deletions(-) diff --git a/src/Core/PostgreSQL/Utils.h b/src/Core/PostgreSQL/Utils.h index b3c035ebac4..34d66fefb70 100644 --- a/src/Core/PostgreSQL/Utils.h +++ b/src/Core/PostgreSQL/Utils.h @@ -11,38 +11,7 @@ namespace pqxx using ReplicationTransaction = pqxx::transaction; } - namespace postgres { - ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password); - -Connection createReplicationConnection(const ConnectionInfo & connection_info); - -template -class Transaction -{ -public: - Transaction(pqxx::connection & connection) : transaction(connection) {} - - ~Transaction() - { - try - { - transaction.commit(); - } - catch (...) - { - DB::tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - T & getRef() { return transaction; } - - void exec(const String & query) { transaction.exec(query); } - -private: - T transaction; -}; - } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 77bef1210ba..3c41ac6da0d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -421,7 +421,6 @@ class IColumn; M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ M(Bool, allow_experimental_database_materialize_postgresql, false, "Allow to create database with Engine=MaterializePostgreSQL(...).", 0) \ - M(Bool, external_databases_use_nulls, true, "If set to false, external databases will use default values instead of NULLs. (Sopported for PostgreSQL/MaterializePostgreSQL database engine)", 0) \ M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 6d4206d1b99..346dca576b2 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -149,9 +149,8 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr if (!table_checked && !checkPostgresTable(table_name)) return StoragePtr{}; - auto use_nulls = local_context->getSettingsRef().external_databases_use_nulls; auto connection_holder = pool->get(); - auto columns = fetchPostgreSQLTableStructure(connection_holder->get(), doubleQuoteString(table_name), use_nulls).columns; + auto columns = fetchPostgreSQLTableStructure(connection_holder->get(), doubleQuoteString(table_name)).columns; if (!columns) return StoragePtr{}; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 199b66b7160..e9edf5b3f5f 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -135,7 +135,7 @@ std::shared_ptr readNamesAndTypesList( std::get<0>(row), /// column name convertPostgreSQLDataType( std::get<1>(row), /// data type - use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false + use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false == nullable std::get<3>(row)))); /// number of dimensions if data type is array } } @@ -213,18 +213,21 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( } -PostgreSQLTableStructure fetchPostgreSQLTableStructure( - pqxx::connection & connection, const String & postgres_table_name, bool use_nulls) +PostgreSQLTableStructure fetchPostgreSQLTableStructure(pqxx::connection & connection, const String & postgres_table_name, bool use_nulls) { - postgres::Transaction tx(connection); - return fetchPostgreSQLTableStructure(tx.getRef(), postgres_table_name, use_nulls, false, false); + pqxx::ReadTransaction tx(connection); + auto result = fetchPostgreSQLTableStructure(tx, postgres_table_name, use_nulls, false, false); + tx.commit(); + return result; } std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection) { - postgres::Transaction tx(connection); - return fetchPostgreSQLTablesList(tx.getRef()); + pqxx::ReadTransaction tx(connection); + auto result = fetchPostgreSQLTablesList(tx); + tx.commit(); + return result; } diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 2b099f02440..07562cd69fa 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -24,11 +24,11 @@ using PostgreSQLTableStructurePtr = std::unique_ptr; std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection); PostgreSQLTableStructure fetchPostgreSQLTableStructure( - pqxx::connection & connection, const String & postgres_table_name, bool use_nulls); + pqxx::connection & connection, const String & postgres_table_name, bool use_nulls = true); template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - T & tx, const String & postgres_table_name, bool use_nulls, + T & tx, const String & postgres_table_name, bool use_nulls = true, bool with_primary_key = false, bool with_replica_identity_index = false); template diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 44eb48deec5..412a38755e3 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -91,12 +91,13 @@ void PostgreSQLReplicationHandler::shutdown() void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { { - postgres::Transaction tx(connection->getRef()); - createPublicationIfNeeded(tx.getRef()); + pqxx::work tx(connection->getRef()); + createPublicationIfNeeded(tx); + tx.commit(); } postgres::Connection replication_connection(connection_info, /* replication */true); - postgres::Transaction tx(replication_connection.getRef()); + pqxx::nontransaction tx(replication_connection.getRef()); /// List of nested tables (table_name -> nested_storage), which is passed to replication consumer. std::unordered_map nested_storages; @@ -112,7 +113,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) auto initial_sync = [&]() { - createReplicationSlot(tx.getRef(), start_lsn, snapshot_name); + createReplicationSlot(tx, start_lsn, snapshot_name); for (const auto & [table_name, storage] : materialized_storages) { @@ -136,7 +137,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. /// Recreation of a replication slot imposes reloading of all tables. - if (!isReplicationSlotExist(tx.getRef(), start_lsn, /* temporary */false)) + if (!isReplicationSlotExist(tx, start_lsn, /* temporary */false)) { initial_sync(); } @@ -145,7 +146,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// TODO: tests else if (new_publication_created) { - dropReplicationSlot(tx.getRef()); + dropReplicationSlot(tx); initial_sync(); } /// Synchronization and initial load already took place - do not create any new tables, just fetch StoragePtr's @@ -187,6 +188,8 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } } + tx.commit(); + /// Pass current connection to consumer. It is not std::moved implicitly, but a shared_ptr is passed. /// Consumer and replication handler are always executed one after another (not concurrently) and share the same connection. /// Handler uses it only for loadFromSnapshot and shutdown methods. @@ -396,34 +399,41 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::shutdownFinal() { - postgres::Transaction tx(connection->getRef()); - dropPublication(tx.getRef()); + pqxx::nontransaction tx(connection->getRef()); + dropPublication(tx); String last_committed_lsn; - if (isReplicationSlotExist(tx.getRef(), last_committed_lsn, /* temporary */false)) - dropReplicationSlot(tx.getRef(), /* temporary */false); - if (isReplicationSlotExist(tx.getRef(), last_committed_lsn, /* temporary */true)) - dropReplicationSlot(tx.getRef(), /* temporary */true); + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */false)) + dropReplicationSlot(tx, /* temporary */false); + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true)) + dropReplicationSlot(tx, /* temporary */true); + tx.commit(); } /// Used by MaterializePostgreSQL database engine. NameSet PostgreSQLReplicationHandler::fetchRequiredTables(pqxx::connection & connection_) { - postgres::Transaction tx(connection_); - bool publication_exists = isPublicationExist(tx.getRef()); + pqxx::work tx(connection_); + bool publication_exists = isPublicationExist(tx); + NameSet result_tables; if (tables_list.empty() && !publication_exists) { /// Fetch all tables list from database. Publication does not exist yet, which means /// that no replication took place. Publication will be created in /// startSynchronization method. - return fetchPostgreSQLTablesList(tx.getRef()); + result_tables = fetchPostgreSQLTablesList(tx); + } + else + { + if (!publication_exists) + createPublicationIfNeeded(tx, /* create_without_check = */ true); + + result_tables = fetchTablesFromPublication(tx); } - if (!publication_exists) - createPublicationIfNeeded(tx.getRef(), /* create_without_check = */ true); - - return fetchTablesFromPublication(tx.getRef()); + tx.commit(); + return result_tables; } @@ -445,8 +455,7 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( if (!is_materialize_postgresql_database) return nullptr; - auto use_nulls = context->getSettingsRef().external_databases_use_nulls; - return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true, true)); + return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, true, true, true)); } @@ -457,12 +466,12 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector tx(replication_connection.getRef()); + pqxx::nontransaction tx(replication_connection.getRef()); String snapshot_name, start_lsn; - if (isReplicationSlotExist(tx.getRef(), start_lsn, /* temporary */true)) - dropReplicationSlot(tx.getRef(), /* temporary */true); - createReplicationSlot(tx.getRef(), start_lsn, snapshot_name, /* temporary */true); + if (isReplicationSlotExist(tx, start_lsn, /* temporary */true)) + dropReplicationSlot(tx, /* temporary */true); + createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true); for (const auto & [relation_id, table_name] : relation_data) { @@ -519,7 +528,8 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector Date: Wed, 12 May 2021 22:47:41 +0000 Subject: [PATCH 087/105] Add replication identifier, allow proper access to _version and _sign columns from nested --- .../DatabaseMaterializePostgreSQL.cpp | 1 + .../PostgreSQLReplicationHandler.cpp | 24 ++++-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 1 + .../StorageMaterializePostgreSQL.cpp | 17 ++++- .../test_storage_postgresql_replica/test.py | 75 +++++++++++++------ 5 files changed, 86 insertions(+), 32 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index 080069c3f44..cbedc98fc3d 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -52,6 +52,7 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( void DatabaseMaterializePostgreSQL::startSynchronization() { replication_handler = std::make_unique( + /* replication_identifier */database_name, remote_database_name, database_name, connection_info, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 412a38755e3..e42e70c50ef 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes } PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( + const String & replication_identifier, const String & remote_database_name_, const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, @@ -41,8 +42,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , tables_list(tables_list_) , connection(std::make_shared(connection_info_)) { - replication_slot = fmt::format("{}_ch_replication_slot", current_database_name); - publication_name = fmt::format("{}_ch_publication", current_database_name); + replication_slot = fmt::format("{}_ch_replication_slot", replication_identifier); + publication_name = fmt::format("{}_ch_publication", replication_identifier); startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); @@ -402,11 +403,20 @@ void PostgreSQLReplicationHandler::shutdownFinal() pqxx::nontransaction tx(connection->getRef()); dropPublication(tx); String last_committed_lsn; - if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */false)) - dropReplicationSlot(tx, /* temporary */false); - if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true)) - dropReplicationSlot(tx, /* temporary */true); - tx.commit(); + try + { + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */false)) + dropReplicationSlot(tx, /* temporary */false); + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true)) + dropReplicationSlot(tx, /* temporary */true); + tx.commit(); + } + catch (Exception & e) + { + e.addMessage("while dropping replication slot: {}", replication_slot); + LOG_ERROR(log, "Failed to drop replication slot: {}. It must be dropped manually.", replication_slot); + throw; + } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 61132d0c0fc..1f8d25ab32d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -19,6 +19,7 @@ class PostgreSQLReplicationHandler { public: PostgreSQLReplicationHandler( + const String & replication_identifier, const String & remote_database_name_, const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 4a7c9655149..806b51bafba 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -63,7 +63,9 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( setInMemoryMetadata(storage_metadata); + String replication_identifier = remote_database_name + "_" + remote_table_name_; replication_handler = std::make_unique( + replication_identifier, remote_database_name, table_id_.database_name, connection_info, @@ -351,11 +353,7 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt "No columns returned for table {}.{}", table_id.database_name, table_id.table_name); } - StorageInMemoryMetadata storage_metadata; - ordinary_columns_and_types = *table_structure->columns; - storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); - setInMemoryMetadata(storage_metadata); if (!table_structure->primary_key_columns && !table_structure->replica_identity_columns) { @@ -406,6 +404,17 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt create_table_query->set(create_table_query->storage, storage); + /// Add columns _sign and _version, so that they can be accessed from nested ReplacingMergeTree table if needed. + /// TODO: add test for case of database engine, test same case after table reload. + ordinary_columns_and_types.push_back({"_sign", std::make_shared()}); + ordinary_columns_and_types.push_back({"_version", std::make_shared()}); + + StorageInMemoryMetadata metadata; + metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); + metadata.setConstraints(metadata_snapshot->getConstraints()); + + setInMemoryMetadata(metadata); + return create_table_query; } diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 20d21008629..53eedbc8b7d 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -77,8 +77,9 @@ def test_initial_load_from_snapshot(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -100,8 +101,9 @@ def test_no_connection_at_startup(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -132,8 +134,9 @@ def test_detach_attach_is_ok(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -167,8 +170,9 @@ def test_replicating_insert_queries(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -208,8 +212,9 @@ def test_replicating_delete_queries(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -246,8 +251,9 @@ def test_replicating_update_queries(started_cluster): instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -276,8 +282,9 @@ def test_resume_from_written_version(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -318,12 +325,9 @@ def test_many_replication_messages(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica ( - key UInt64, value UInt64, - _sign Int8 MATERIALIZED 1, - _version UInt64 MATERIALIZED 1, - PRIMARY KEY(key)) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, PRIMARY KEY(key)) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') SETTINGS materialize_postgresql_max_block_size = 50000; @@ -376,8 +380,9 @@ def test_connection_loss(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; @@ -412,8 +417,9 @@ def test_clickhouse_restart(started_cluster): create_postgres_table(cursor, 'postgresql_replica'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') @@ -439,32 +445,59 @@ def test_rename_table(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() create_postgres_table(cursor, 'postgresql_replica'); - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL( 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; ''') + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)") + result = instance.query('SELECT count() FROM test.postgresql_replica;') - while int(result) != 50: + while int(result) != 25: time.sleep(0.5) result = instance.query('SELECT count() FROM test.postgresql_replica;') instance.query('RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed') + assert(int(instance.query('SELECT count() FROM test.postgresql_replica_renamed;')) == 25) + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25, 25)") result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') while int(result) != 50: time.sleep(0.5) result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') - instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50, 50)") + result = instance.query('SELECT * FROM test.postgresql_replica_renamed ORDER BY key;') + postgresql_replica_check_result(result, True) + cursor.execute('DROP TABLE postgresql_replica;') + instance.query('DROP TABLE IF EXISTS test.postgresql_replica_renamed') - result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') - while int(result) != 100: + +def test_virtual_columns(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica'); + + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + ENGINE = MaterializePostgreSQL( + 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; ''') + + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)") + result = instance.query('SELECT count() FROM test.postgresql_replica;') + while int(result) != 10: time.sleep(0.5) - result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;') + result = instance.query('SELECT count() FROM test.postgresql_replica;') + + # just check that it works, no check with `expected` becuase _version is taken as LSN, which will be different each time. + result = instance.query('SELECT key, value, _sign, _version FROM test.postgresql_replica;') + print(result) + cursor.execute('DROP TABLE postgresql_replica;') if __name__ == '__main__': From 85bbfd54e818b21ddf19556770b11dbbe5fe781d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 May 2021 07:36:40 +0000 Subject: [PATCH 088/105] Fix bug --- .../integrations/materialize-postgresql.md | 13 ++++--- .../MaterializePostgreSQLConsumer.cpp | 5 ++- .../MaterializePostgreSQLConsumer.h | 3 ++ .../PostgreSQLReplicationHandler.cpp | 11 +++++- .../StorageMaterializePostgreSQL.cpp | 1 - .../test.py | 35 ++++++++++++++++++- 6 files changed, 60 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/materialize-postgresql.md b/docs/en/engines/table-engines/integrations/materialize-postgresql.md index c40ea6b72db..aba1a370792 100644 --- a/docs/en/engines/table-engines/integrations/materialize-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialize-postgresql.md @@ -8,7 +8,7 @@ toc_title: MateriaziePostgreSQL ## Creating a Table {#creating-a-table} ``` sql -CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) +CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') PRIMARY KEY key; ``` @@ -25,12 +25,17 @@ PRIMARY KEY key; ## Virtual columns {#creating-a-table} -- `_version` +- `_version` (`UInt64`) -- `_sign` +- `_sign` (`Int8`) + +These columns do not need to be added, when table is created. They are always accessible in `SELECT` query. +`_version` column equals `LSN` position in `WAL`, so it might be used to check how up-to-date replication is. ``` sql -CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1) +CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') PRIMARY KEY key; + +SELECT key, value, _version FROM test.postgresql_replica; ``` diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 5e41e20550e..5c4fc27a334 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -51,6 +51,9 @@ void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage { const auto storage_metadata = storage->getInMemoryMetadataPtr(); const Block sample_block = storage_metadata->getSampleBlock(); + + /// Need to clear type, because in description.init() the types are appended (emplace_back) + description.types.clear(); description.init(sample_block); columns = description.sample_block.cloneEmptyColumns(); @@ -560,7 +563,7 @@ void MaterializePostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const if (allow_automatic_update) LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id); else - LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name); + LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name, relation_id); } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h index 43ac919f119..00523ff0ea9 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h @@ -51,7 +51,10 @@ private: { ExternalResultDescription description; MutableColumns columns; + + /// Needed to pass to insert query columns list in syncTables(). std::shared_ptr columnsAST; + /// Needed for insertPostgreSQLValue() method to parse array std::unordered_map array_info; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index e42e70c50ef..5f1a62d8086 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -523,7 +523,16 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorsetNestedStorageID(nested_table_id); nested_storage = materialized_storage->prepare(); - LOG_TRACE(log, "Updated table {}.{} ({})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); + + auto nested_storage_metadata = nested_storage->getInMemoryMetadataPtr(); + auto nested_sample_block = nested_storage_metadata->getSampleBlock(); + LOG_TRACE(log, "Updated table {}.{} ({}). New structure: {}", + nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid), nested_sample_block.dumpStructure()); + + auto materialized_storage_metadata = nested_storage->getInMemoryMetadataPtr(); + auto materialized_sample_block = materialized_storage_metadata->getSampleBlock(); + + assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 806b51bafba..7c20b49897f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -405,7 +405,6 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt create_table_query->set(create_table_query->storage, storage); /// Add columns _sign and _version, so that they can be accessed from nested ReplacingMergeTree table if needed. - /// TODO: add test for case of database engine, test same case after table reload. ordinary_columns_and_types.push_back({"_sign", std::make_shared()}); ordinary_columns_and_types.push_back({"_version", std::make_shared()}); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index f16e0125efd..3526cac57e7 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -555,7 +555,40 @@ def test_single_transaction(started_cluster): assert(int(result) == 0) conn.commit() - check_tables_are_synchronized('postgresql_replica_{}'.format(0)); + check_tables_are_synchronized('postgresql_replica_0'); + instance.query("DROP DATABASE test_database") + + +def test_virtual_columns(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + create_postgres_table(cursor, 'postgresql_replica_0'); + + instance.query( + """CREATE DATABASE test_database + ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') + SETTINGS materialize_postgresql_allow_automatic_update = 1; """) + assert_nested_table_is_created('postgresql_replica_0') + instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number from numbers(10)") + check_tables_are_synchronized('postgresql_replica_0'); + + # just check that it works, no check with `expected` becuase _version is taken as LSN, which will be different each time. + result = instance.query('SELECT key, value, _sign, _version FROM test_database.postgresql_replica_0;') + print(result) + + cursor.execute("ALTER TABLE postgresql_replica_0 ADD COLUMN value2 integer") + instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number, number from numbers(10, 10)") + check_tables_are_synchronized('postgresql_replica_0'); + + result = instance.query('SELECT key, value, value2, _sign, _version FROM test_database.postgresql_replica_0;') + print(result) + + instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number, number from numbers(20, 10)") + check_tables_are_synchronized('postgresql_replica_0'); + + result = instance.query('SELECT key, value, value2, _sign, _version FROM test_database.postgresql_replica_0;') + print(result) + instance.query("DROP DATABASE test_database") From 1ee77eae8982f7859c07c899423309cad155fd65 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 May 2021 21:35:37 +0000 Subject: [PATCH 089/105] Better way to distinguish between user query and replication thread query --- .../PostgreSQL/DatabaseMaterializePostgreSQL.cpp | 13 ++++--------- src/Interpreters/Context.h | 6 ++++++ .../PostgreSQL/MaterializePostgreSQLConsumer.cpp | 3 +-- .../PostgreSQL/StorageMaterializePostgreSQL.cpp | 4 +--- 4 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp index cbedc98fc3d..3cb5b7d7c55 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp @@ -118,8 +118,7 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte /// to its nested ReplacingMergeTree table (in all other cases), the context of a query os modified. /// Also if materialzied_tables set is empty - it means all access is done to ReplacingMergeTree tables - it is a case after /// replication_handler was shutdown. - if ((local_context->hasQueryContext() && local_context->getQueryContext()->getQueryFactoriesInfo().storages.count("ReplacingMergeTree")) - || materialized_tables.empty()) + if (local_context->isInternalQuery() || materialized_tables.empty()) { return DatabaseAtomic::tryGetTable(name, local_context); } @@ -143,14 +142,10 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) { /// Create table query can only be called from replication thread. - if (local_context->hasQueryContext()) + if (local_context->isInternalQuery()) { - auto storage_set = local_context->getQueryContext()->getQueryFactoriesInfo().storages; - if (storage_set.find("ReplacingMergeTree") != storage_set.end()) - { - DatabaseAtomic::createTable(local_context, table_name, table, query); - return; - } + DatabaseAtomic::createTable(local_context, table_name, table, query); + return; } throw Exception(ErrorCodes::NOT_IMPLEMENTED, diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a8fd0cf1700..de1d9b94ca7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -259,6 +259,9 @@ private: /// XXX: move this stuff to shared part instead. ContextPtr buffer_context; /// Buffer context. Could be equal to this. + /// A flag, used to distinquish between user query and internal query to a database engine (MaterializePostgreSQL). + bool is_internal_query = false; + public: // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. OpenTelemetryTraceContext query_trace_context; @@ -728,6 +731,9 @@ public: void shutdown(); + bool isInternalQuery() const { return is_internal_query; } + void setInternalQuery(bool internal) { is_internal_query = internal; } + ActionLocksManagerPtr getActionLocksManager(); enum class ApplicationType diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index 5c4fc27a334..418a95ae078 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -475,8 +475,7 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptrcolumns = buffer.columnsAST; auto insert_context = Context::createCopy(context); - insert_context->makeQueryContext(); - insert_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); + insert_context->setInternalQuery(true); InterpreterInsertQuery interpreter(insert, insert_context, true); auto block_io = interpreter.execute(); diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 7c20b49897f..1b99b2eabfb 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -173,9 +173,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) { auto new_context = Context::createCopy(from_context); - new_context->makeQueryContext(); - new_context->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, "ReplacingMergeTree"); - + new_context->setInternalQuery(true); return new_context; } From 9758e46c3c8978ed25b8d296e59905b665699811 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 21 May 2021 17:46:28 +0300 Subject: [PATCH 090/105] Add TODO --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 1f8d25ab32d..db29a2b4aaa 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -5,6 +5,10 @@ #include +/// TODO: Add test for multiple databases to be replicated. Add test to make sure unneeded tables fo not get into replication slot. +/// Test behavior of publication. + + namespace DB { From da32f661228ea9bf516ee418b7270f4c4fedfd05 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 23 May 2021 12:09:20 +0000 Subject: [PATCH 091/105] More tests --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 2 +- .../PostgreSQLReplicationHandler.cpp | 18 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 - .../test.py | 213 +++++++++++++----- 4 files changed, 161 insertions(+), 76 deletions(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 07f2404fdc3..26138bafb92 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -114,7 +114,7 @@ void insertPostgreSQLValue( size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions; const auto parse_value = array_info[idx].pqxx_parser; - std::vector> dimensions(expected_dimensions + 1); + std::vector dimensions(expected_dimensions + 1); while (parsed.first != pqxx::array_parser::juncture::done) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 5f1a62d8086..ac9ff2edbe1 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -165,21 +165,6 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } catch (Exception & e) { - if (e.code() == ErrorCodes::UNKNOWN_TABLE) - { - try - { - /// If nested table does not exist, try load it once again. - loadFromSnapshot(snapshot_name, table_name, storage->as ()); - nested_storages[table_name] = materialized_storage->prepare(); - continue; - } - catch (...) - { - e.addMessage("Table load failed for the second time"); - } - } - e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); @@ -309,6 +294,9 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bo } } + if (tables_list.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No table found to be replicated"); + /// 'ONLY' means just a table, without descendants. std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, tables_list); try diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index db29a2b4aaa..1f8d25ab32d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -5,10 +5,6 @@ #include -/// TODO: Add test for multiple databases to be replicated. Add test to make sure unneeded tables fo not get into replication slot. -/// Test behavior of publication. - - namespace DB { diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 3526cac57e7..c98e4ee14d8 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -31,9 +31,9 @@ postgres_table_template_3 = """ key1 Integer NOT NULL, value1 Integer, key2 Integer NOT NULL, value2 Integer NOT NULL) """ -def get_postgres_conn(database=False, auto_commit=True): +def get_postgres_conn(database=False, auto_commit=True, database_name='postgres_database'): if database == True: - conn_string = "host='localhost' dbname='postgres_database' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' dbname='{}' user='postgres' password='mysecretpassword'".format(database_name) else: conn_string = "host='localhost' user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) @@ -43,9 +43,27 @@ def get_postgres_conn(database=False, auto_commit=True): return conn -def create_postgres_db(cursor, name): +def create_postgres_db(cursor, name='postgres_database'): cursor.execute("CREATE DATABASE {}".format(name)) +def drop_postgres_db(cursor, name='postgres_database'): + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + +def create_clickhouse_postgres_db(name='postgres_database'): + instance.query(''' + CREATE DATABASE {} + ENGINE = PostgreSQL('postgres1:5432', '{}', 'postgres', 'mysecretpassword')'''.format(name, name)) + +def drop_clickhouse_postgres_db(name='postgres_database'): + instance.query('DROP DATABASE IF EXISTS {}'.format(name)) + +def create_materialized_db(materialized_database='test_database', postgres_database='postgres_database'): + instance.query("CREATE DATABASE {} ENGINE = MaterializePostgreSQL('postgres1:5432', '{}', 'postgres', 'mysecretpassword')".format(materialized_database, postgres_database)) + assert materialized_database in instance.query('SHOW DATABASES') + +def drop_materialized_db(materialized_database='test_database'): + instance.query('DROP DATABASE IF EXISTS {}'.format(materialized_database)) + assert materialized_database not in instance.query('SHOW DATABASES') def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template): cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) @@ -53,26 +71,47 @@ def create_postgres_table(cursor, table_name, replica_identity_full=False, templ if replica_identity_full: cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) +queries = [ + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(0, 10000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', + 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;', + "UPDATE postgresql_replica_{} SET key=key+20000 WHERE key%2=0", + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(40000, 50000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', + 'UPDATE postgresql_replica_{} SET value = value + 101 WHERE key % 2 = 1;', + "UPDATE postgresql_replica_{} SET key=key+80000 WHERE key%2=1", + 'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;', + 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;', + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(200000, 250000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;', + 'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;', + "UPDATE postgresql_replica_{} SET key=key+500000 WHERE key%2=1", + 'INSERT INTO postgresql_replica_{} select i, i from generate_series(1000000, 1050000) as t(i);', + 'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;', + "UPDATE postgresql_replica_{} SET key=key+10000000", + 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;', + 'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;'] + @pytest.mark.timeout(30) -def assert_nested_table_is_created(table_name): - database_tables = instance.query('SHOW TABLES FROM test_database') +def assert_nested_table_is_created(table_name, materialized_database='test_database'): + database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database)) while table_name not in database_tables: time.sleep(0.2) - database_tables = instance.query('SHOW TABLES FROM test_database') + database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database)) assert(table_name in database_tables) @pytest.mark.timeout(30) -def check_tables_are_synchronized(table_name, order_by='key'): - assert_nested_table_is_created(table_name) +def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'): + assert_nested_table_is_created(table_name, materialized_database) - expected = instance.query('select * from postgres_database.{} order by {};'.format(table_name, order_by)) - result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) + expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) + result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) while result != expected: time.sleep(0.5) - result = instance.query('select * from test_database.{} order by {};'.format(table_name, order_by)) + result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) assert(result == expected) @@ -84,10 +123,9 @@ def started_cluster(): conn = get_postgres_conn() cursor = conn.cursor() create_postgres_db(cursor, 'postgres_database') + create_clickhouse_postgres_db() + instance.query("DROP DATABASE IF EXISTS test_database") - instance.query(''' - CREATE DATABASE postgres_database - ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')''') yield cluster finally: @@ -432,7 +470,7 @@ def test_table_schema_changes(started_cluster): @pytest.mark.timeout(120) -def test_random_queries(started_cluster): +def test_many_concurrent_queries(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") conn = get_postgres_conn(True) cursor = conn.cursor() @@ -443,7 +481,7 @@ def test_random_queries(started_cluster): instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(10000)'.format(i)) n = [10000] - query = ['DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', + query_pool = ['DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;', 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', 'UPDATE postgresql_replica_{} SET value = value*5 WHERE key % 2 = 1;', @@ -458,12 +496,12 @@ def test_random_queries(started_cluster): def attack(thread_id): print('thread {}'.format(thread_id)) k = 10000 - for i in range(10): - query_id = random.randrange(0, len(query)-1) + for i in range(20): + query_id = random.randrange(0, len(query_pool)-1) table_id = random.randrange(0, 5) # num tables # random update / delete query - cursor.execute(query[query_id].format(table_id)) + cursor.execute(query_pool[query_id].format(table_id)) print("table {} query {} ok".format(table_id, query_id)) # allow some thread to do inserts (not to violate key constraints) @@ -481,16 +519,15 @@ def test_random_queries(started_cluster): threads = [] threads_num = 16 - for i in range(threads_num): threads.append(threading.Thread(target=attack, args=(i,))) + + create_materialized_db() + for thread in threads: time.sleep(random.uniform(0, 1)) thread.start() - instance.query( - "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") - n[0] = 50000 for table_id in range(NUM_TABLES): n[0] += 1 @@ -502,14 +539,11 @@ def test_random_queries(started_cluster): for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - count = instance.query('SELECT count() FROM test_database.postgresql_replica_{}'.format(i)) - print(count) - - for i in range(NUM_TABLES): - cursor.execute('drop table postgresql_replica_{};'.format(i)) - - instance.query("DROP DATABASE test_database") - assert 'test_database' not in instance.query('SHOW DATABASES') + count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i)) + count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i)) + assert(int(count1) == int(count2)) + print(count1, count2) + drop_materialized_db() @pytest.mark.timeout(120) @@ -520,30 +554,9 @@ def test_single_transaction(started_cluster): create_postgres_table(cursor, 'postgresql_replica_0'); conn.commit() - instance.query( - "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") - assert_nested_table_is_created('postgresql_replica_0') - queries = [ - 'INSERT INTO postgresql_replica_{} select i, i from generate_series(0, 10000) as t(i);', - 'DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', - 'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;', - "UPDATE postgresql_replica_{} SET key=key+20000 WHERE key%2=0", - 'INSERT INTO postgresql_replica_{} select i, i from generate_series(40000, 50000) as t(i);', - 'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;', - 'UPDATE postgresql_replica_{} SET value = value + 101 WHERE key % 2 = 1;', - "UPDATE postgresql_replica_{} SET key=key+80000 WHERE key%2=1", - 'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;', - 'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;', - 'INSERT INTO postgresql_replica_{} select i, i from generate_series(200000, 250000) as t(i);', - 'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;', - 'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;', - "UPDATE postgresql_replica_{} SET key=key+500000 WHERE key%2=1", - 'INSERT INTO postgresql_replica_{} select i, i from generate_series(1000000, 1050000) as t(i);', - 'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;', - "UPDATE postgresql_replica_{} SET key=key+10000000", - 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;', - 'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;'] + create_materialized_db() + assert_nested_table_is_created('postgresql_replica_0') for query in queries: print('query {}'.format(query)) @@ -556,7 +569,7 @@ def test_single_transaction(started_cluster): conn.commit() check_tables_are_synchronized('postgresql_replica_0'); - instance.query("DROP DATABASE test_database") + drop_materialized_db() def test_virtual_columns(started_cluster): @@ -588,8 +601,96 @@ def test_virtual_columns(started_cluster): result = instance.query('SELECT key, value, value2, _sign, _version FROM test_database.postgresql_replica_0;') print(result) + drop_materialized_db() - instance.query("DROP DATABASE test_database") + +def test_multiple_databases(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database_1") + instance.query("DROP DATABASE IF EXISTS test_database_2") + NUM_TABLES = 5 + + conn = get_postgres_conn() + cursor = conn.cursor() + create_postgres_db(cursor, 'postgres_database_1') + create_postgres_db(cursor, 'postgres_database_2') + + conn1 = get_postgres_conn(True, True, 'postgres_database_1') + conn2 = get_postgres_conn(True, True, 'postgres_database_2') + + cursor1 = conn1.cursor() + cursor2 = conn2.cursor() + + create_clickhouse_postgres_db('postgres_database_1') + create_clickhouse_postgres_db('postgres_database_2') + + cursors = [cursor1, cursor2] + for cursor_id in range(len(cursors)): + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table(cursors[cursor_id], table_name); + instance.query("INSERT INTO postgres_database_{}.{} SELECT number, number from numbers(50)".format(cursor_id + 1, table_name)) + print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';''')) + print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';''')) + + create_materialized_db('test_database_1', 'postgres_database_1') + create_materialized_db('test_database_2', 'postgres_database_2') + + cursors = [cursor1, cursor2] + for cursor_id in range(len(cursors)): + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + instance.query("INSERT INTO postgres_database_{}.{} SELECT 50 + number, number from numbers(50)".format(cursor_id + 1, table_name)) + + for cursor_id in range(len(cursors)): + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + check_tables_are_synchronized( + table_name, 'key', 'postgres_database_{}'.format(cursor_id + 1), 'test_database_{}'.format(cursor_id + 1)); + + drop_clickhouse_postgres_db('postgres_database_1') + drop_clickhouse_postgres_db('postgres_database_2') + drop_materialized_db('test_database_1') + drop_materialized_db('test_database_2') + + +@pytest.mark.timeout(320) +def test_concurrent_transactions(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 6 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + + def transaction(thread_id): + conn_ = get_postgres_conn(True, auto_commit=False) + cursor_ = conn.cursor() + for query in queries: + cursor_.execute(query.format(thread_id)) + print('thread {}, query {}'.format(thread_id, query)) + conn_.commit() + + threads = [] + threads_num = 6 + for i in range(threads_num): + threads.append(threading.Thread(target=transaction, args=(i,))) + + create_materialized_db() + + for thread in threads: + time.sleep(random.uniform(0, 0.5)) + thread.start() + for thread in threads: + thread.join() + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i)) + count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i)) + print(int(count1), int(count2), sep=' ') + assert(int(count1) == int(count2)) + drop_materialized_db() if __name__ == '__main__': From 625377f553d8aca133a512ed0000a5bba589a0e9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 26 May 2021 23:38:53 +0000 Subject: [PATCH 092/105] Add backoff for reading replication messages, fix some checks --- src/Interpreters/Context.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 19 ++++++++++++++----- .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 +++- .../StorageMaterializePostgreSQL.cpp | 8 ++++---- 4 files changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b12d4c82ce8..5e9a84516ef 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -259,7 +259,7 @@ private: /// XXX: move this stuff to shared part instead. ContextPtr buffer_context; /// Buffer context. Could be equal to this. - /// A flag, used to distinquish between user query and internal query to a database engine (MaterializePostgreSQL). + /// A flag, used to distinguish between user query and internal query to a database engine (MaterializePostgreSQL). bool is_internal_query = false; public: diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index ac9ff2edbe1..33d5c49ec09 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -14,11 +14,12 @@ namespace DB { -static const auto reschedule_ms = 500; +static const auto RESCHEDULE_MS = 500; +static const auto BACKOFF_TRESHOLD = 32000; namespace ErrorCodes { - extern const int UNKNOWN_TABLE; + extern const int LOGICAL_ERROR; } PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( @@ -41,6 +42,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , is_materialize_postgresql_database(is_materialize_postgresql_database_) , tables_list(tables_list_) , connection(std::make_shared(connection_info_)) + , milliseconds_to_wait(RESCHEDULE_MS) { replication_slot = fmt::format("{}_ch_replication_slot", replication_identifier); publication_name = fmt::format("{}_ch_publication", replication_identifier); @@ -72,7 +74,7 @@ void PostgreSQLReplicationHandler::waitConnectionAndStart() catch (const pqxx::broken_connection & pqxx_error) { 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(RESCHEDULE_MS); } catch (...) { @@ -256,9 +258,16 @@ void PostgreSQLReplicationHandler::consumerFunc() return; if (schedule_now) + { consumer_task->schedule(); + milliseconds_to_wait = RESCHEDULE_MS; + } else - consumer_task->scheduleAfter(reschedule_ms); + { + consumer_task->scheduleAfter(milliseconds_to_wait); + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; + } } @@ -448,7 +457,7 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( - pqxx::ReplicationTransaction & tx, const std::string & table_name) + pqxx::ReplicationTransaction & tx, const std::string & table_name) const { if (!is_materialize_postgresql_database) return nullptr; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 1f8d25ab32d..6c919389392 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -78,7 +78,7 @@ private: void reloadFromSnapshot(const std::vector> & relation_data); - PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name); + PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name) const; Poco::Logger * log; ContextPtr context; @@ -120,6 +120,8 @@ private: /// MaterializePostgreSQL tables. Used for managing all operations with its internal nested tables. MaterializedStorages materialized_storages; + + UInt64 milliseconds_to_wait; }; } diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp index 1b99b2eabfb..07d13ace7c2 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp @@ -406,11 +406,11 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt ordinary_columns_and_types.push_back({"_sign", std::make_shared()}); ordinary_columns_and_types.push_back({"_version", std::make_shared()}); - StorageInMemoryMetadata metadata; - metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); - metadata.setConstraints(metadata_snapshot->getConstraints()); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); + storage_metadata.setConstraints(metadata_snapshot->getConstraints()); - setInMemoryMetadata(metadata); + setInMemoryMetadata(storage_metadata); return create_table_query; } From 00e76ca372edccde2b6f7ac7430d3231878b19e8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Jun 2021 19:45:27 +0000 Subject: [PATCH 093/105] Checking tests --- src/Storages/StorageMaterializeMySQL.cpp | 66 +++- .../test.py | 303 ++++++++++-------- .../test_storage_postgresql_replica/test.py | 176 +++++----- 3 files changed, 307 insertions(+), 238 deletions(-) diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 6352b62d6f4..8e6f2e1ad63 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -23,7 +23,6 @@ #include #include -#include namespace DB { @@ -38,7 +37,7 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora Pipe StorageMaterializeMySQL::read( const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, + const StorageMetadataPtr & /*metadata_snapshot*/, SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum processed_stage, @@ -48,15 +47,60 @@ Pipe StorageMaterializeMySQL::read( /// If the background synchronization thread has exception. rethrowSyncExceptionIfNeed(database); - return readFinalFromNestedStorage( - nested_storage, - column_names, - metadata_snapshot, - query_info, - context, - processed_stage, - max_block_size, - num_streams); + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); + auto lock = nested_storage->lockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); + const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); + + Block nested_header = nested_metadata->getSampleBlock(); + ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); + ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + + if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) + { + auto & tables_in_select_query = select_query->tables()->as(); + + if (!tables_in_select_query.children.empty()) + { + auto & tables_element = tables_in_select_query.children[0]->as(); + + if (tables_element.table_expression) + tables_element.table_expression->as().final = true; + } + } + + String filter_column_name; + Names require_columns_name = column_names; + ASTPtr expressions = std::make_shared(); + if (column_names_set.empty() || !column_names_set.count(sign_column.name)) + { + require_columns_name.emplace_back(sign_column.name); + + const auto & sign_column_name = std::make_shared(sign_column.name); + const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); + + expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); + filter_column_name = expressions->children.back()->getColumnName(); + + for (const auto & column_name : column_names) + expressions->children.emplace_back(std::make_shared(column_name)); + } + + Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); + pipe.addTableLock(lock); + + if (!expressions->children.empty() && !pipe.empty()) + { + Block pipe_header = pipe.getHeader(); + auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true /* add_aliases */, false /* project_result */); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, expression_actions, filter_column_name, false); + }); + } + + return pipe; } NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index c98e4ee14d8..f19a5cf2467 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -31,11 +31,12 @@ postgres_table_template_3 = """ key1 Integer NOT NULL, value1 Integer, key2 Integer NOT NULL, value2 Integer NOT NULL) """ -def get_postgres_conn(database=False, auto_commit=True, database_name='postgres_database'): +def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database'): if database == True: - conn_string = "host='localhost' dbname='{}' user='postgres' password='mysecretpassword'".format(database_name) + conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format(ip, port, database_name) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port) + conn = psycopg2.connect(conn_string) if auto_commit: conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) @@ -47,22 +48,32 @@ def create_postgres_db(cursor, name='postgres_database'): cursor.execute("CREATE DATABASE {}".format(name)) def drop_postgres_db(cursor, name='postgres_database'): - cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("DROP DATABASE {}".format(name)) -def create_clickhouse_postgres_db(name='postgres_database'): +def create_clickhouse_postgres_db(ip, port, name='postgres_database'): instance.query(''' CREATE DATABASE {} - ENGINE = PostgreSQL('postgres1:5432', '{}', 'postgres', 'mysecretpassword')'''.format(name, name)) + ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')'''.format(name, ip, port, name)) def drop_clickhouse_postgres_db(name='postgres_database'): - instance.query('DROP DATABASE IF EXISTS {}'.format(name)) + instance.query('DROP DATABASE {}'.format(name)) -def create_materialized_db(materialized_database='test_database', postgres_database='postgres_database'): - instance.query("CREATE DATABASE {} ENGINE = MaterializePostgreSQL('postgres1:5432', '{}', 'postgres', 'mysecretpassword')".format(materialized_database, postgres_database)) +def create_materialized_db(ip, port, + materialized_database='test_database', + postgres_database='postgres_database', + settings=[]): + create_query = "CREATE DATABASE {} ENGINE = MaterializePostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')".format(materialized_database, ip, port, postgres_database) + if len(settings) > 0: + create_query += " SETTINGS " + for i in range(len(settings)): + if i != 0: + create_query += ', ' + create_query += settings[i] + instance.query(create_query) assert materialized_database in instance.query('SHOW DATABASES') def drop_materialized_db(materialized_database='test_database'): - instance.query('DROP DATABASE IF EXISTS {}'.format(materialized_database)) + instance.query('DROP DATABASE {}'.format(materialized_database)) assert materialized_database not in instance.query('SHOW DATABASES') def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template): @@ -120,10 +131,12 @@ def check_tables_are_synchronized(table_name, order_by='key', postgres_database= def started_cluster(): try: cluster.start() - conn = get_postgres_conn() + conn = get_postgres_conn(ip=cluster.postgres_ip, + port=cluster.postgres_port) cursor = conn.cursor() create_postgres_db(cursor, 'postgres_database') - create_clickhouse_postgres_db() + create_clickhouse_postgres_db(ip=cluster.postgres_ip, + port=cluster.postgres_port) instance.query("DROP DATABASE IF EXISTS test_database") yield cluster @@ -140,7 +153,9 @@ def postgresql_setup_teardown(): @pytest.mark.timeout(120) def test_load_and_sync_all_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 5 @@ -149,7 +164,8 @@ def test_load_and_sync_all_database_tables(started_cluster): create_postgres_table(cursor, table_name); instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(50)".format(table_name)) - instance.query("CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) assert 'test_database' in instance.query('SHOW DATABASES') for i in range(NUM_TABLES): @@ -167,7 +183,9 @@ def test_load_and_sync_all_database_tables(started_cluster): @pytest.mark.timeout(120) def test_replicating_dml(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 5 @@ -175,8 +193,8 @@ def test_replicating_dml(started_cluster): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i)) - instance.query( - "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) for i in range(NUM_TABLES): instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format(i, i)) @@ -210,7 +228,9 @@ def test_replicating_dml(started_cluster): @pytest.mark.timeout(120) def test_different_data_types(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() cursor.execute('drop table if exists test_data_types;') cursor.execute('drop table if exists test_array_data_type;') @@ -236,8 +256,8 @@ def test_different_data_types(started_cluster): k Char(2)[] -- Nullable(String) )''') - instance.query( - "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) for i in range(10): instance.query(''' @@ -294,7 +314,9 @@ def test_different_data_types(started_cluster): @pytest.mark.timeout(120) def test_load_and_sync_subset_of_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 10 @@ -309,11 +331,9 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): publication_tables += ', ' publication_tables += table_name - instance.query(''' - CREATE DATABASE test_database - ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS materialize_postgresql_tables_list = '{}'; - '''.format(publication_tables)) + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=["materialize_postgresql_tables_list = '{}'".format(publication_tables)]) assert 'test_database' in instance.query('SHOW DATABASES') time.sleep(1) @@ -347,13 +367,15 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): @pytest.mark.timeout(120) def test_changing_replica_identity_value(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)") - instance.query( - "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)") check_tables_are_synchronized('postgresql_replica'); @@ -364,7 +386,9 @@ def test_changing_replica_identity_value(started_cluster): @pytest.mark.timeout(320) def test_clickhouse_restart(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 5 @@ -390,7 +414,9 @@ def test_clickhouse_restart(started_cluster): @pytest.mark.timeout(120) def test_replica_identity_index(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + 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', template=postgres_table_template_3); @@ -398,8 +424,8 @@ def test_replica_identity_index(started_cluster): 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)") - instance.query( - "CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(100, 10)") check_tables_are_synchronized('postgresql_replica', order_by='key1'); @@ -416,7 +442,9 @@ def test_replica_identity_index(started_cluster): @pytest.mark.timeout(320) def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 5 @@ -424,11 +452,9 @@ def test_table_schema_changes(started_cluster): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i), template=postgres_table_template_2); instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i)) - instance.query( - """CREATE DATABASE test_database - ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS materialize_postgresql_allow_automatic_update = 1; - """) + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=["materialize_postgresql_allow_automatic_update = 1"]) 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)) @@ -472,7 +498,9 @@ def test_table_schema_changes(started_cluster): @pytest.mark.timeout(120) def test_many_concurrent_queries(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 5 @@ -522,7 +550,8 @@ def test_many_concurrent_queries(started_cluster): for i in range(threads_num): threads.append(threading.Thread(target=attack, args=(i,))) - create_materialized_db() + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) for thread in threads: time.sleep(random.uniform(0, 1)) @@ -549,13 +578,16 @@ def test_many_concurrent_queries(started_cluster): @pytest.mark.timeout(120) def test_single_transaction(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(database=True, auto_commit=False) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=False) cursor = conn.cursor() create_postgres_table(cursor, 'postgresql_replica_0'); conn.commit() - create_materialized_db() + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) assert_nested_table_is_created('postgresql_replica_0') for query in queries: @@ -573,14 +605,15 @@ def test_single_transaction(started_cluster): def test_virtual_columns(started_cluster): - conn = get_postgres_conn(True) + 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_0'); - instance.query( - """CREATE DATABASE test_database - ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword') - SETTINGS materialize_postgresql_allow_automatic_update = 1; """) + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=["materialize_postgresql_allow_automatic_update = 1"]) assert_nested_table_is_created('postgresql_replica_0') instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number from numbers(10)") check_tables_are_synchronized('postgresql_replica_0'); @@ -604,93 +637,93 @@ def test_virtual_columns(started_cluster): drop_materialized_db() -def test_multiple_databases(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database_1") - instance.query("DROP DATABASE IF EXISTS test_database_2") - NUM_TABLES = 5 - - conn = get_postgres_conn() - cursor = conn.cursor() - create_postgres_db(cursor, 'postgres_database_1') - create_postgres_db(cursor, 'postgres_database_2') - - conn1 = get_postgres_conn(True, True, 'postgres_database_1') - conn2 = get_postgres_conn(True, True, 'postgres_database_2') - - cursor1 = conn1.cursor() - cursor2 = conn2.cursor() - - create_clickhouse_postgres_db('postgres_database_1') - create_clickhouse_postgres_db('postgres_database_2') - - cursors = [cursor1, cursor2] - for cursor_id in range(len(cursors)): - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - create_postgres_table(cursors[cursor_id], table_name); - instance.query("INSERT INTO postgres_database_{}.{} SELECT number, number from numbers(50)".format(cursor_id + 1, table_name)) - print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';''')) - print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';''')) - - create_materialized_db('test_database_1', 'postgres_database_1') - create_materialized_db('test_database_2', 'postgres_database_2') - - cursors = [cursor1, cursor2] - for cursor_id in range(len(cursors)): - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - instance.query("INSERT INTO postgres_database_{}.{} SELECT 50 + number, number from numbers(50)".format(cursor_id + 1, table_name)) - - for cursor_id in range(len(cursors)): - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - check_tables_are_synchronized( - table_name, 'key', 'postgres_database_{}'.format(cursor_id + 1), 'test_database_{}'.format(cursor_id + 1)); - - drop_clickhouse_postgres_db('postgres_database_1') - drop_clickhouse_postgres_db('postgres_database_2') - drop_materialized_db('test_database_1') - drop_materialized_db('test_database_2') - - -@pytest.mark.timeout(320) -def test_concurrent_transactions(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) - cursor = conn.cursor() - NUM_TABLES = 6 - - for i in range(NUM_TABLES): - create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); - - def transaction(thread_id): - conn_ = get_postgres_conn(True, auto_commit=False) - cursor_ = conn.cursor() - for query in queries: - cursor_.execute(query.format(thread_id)) - print('thread {}, query {}'.format(thread_id, query)) - conn_.commit() - - threads = [] - threads_num = 6 - for i in range(threads_num): - threads.append(threading.Thread(target=transaction, args=(i,))) - - create_materialized_db() - - for thread in threads: - time.sleep(random.uniform(0, 0.5)) - thread.start() - for thread in threads: - thread.join() - - for i in range(NUM_TABLES): - check_tables_are_synchronized('postgresql_replica_{}'.format(i)); - count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i)) - count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i)) - print(int(count1), int(count2), sep=' ') - assert(int(count1) == int(count2)) - drop_materialized_db() +#def test_multiple_databases(started_cluster): +# instance.query("DROP DATABASE IF EXISTS test_database_1") +# instance.query("DROP DATABASE IF EXISTS test_database_2") +# NUM_TABLES = 5 +# +# conn = get_postgres_conn() +# cursor = conn.cursor() +# create_postgres_db(cursor, 'postgres_database_1') +# create_postgres_db(cursor, 'postgres_database_2') +# +# conn1 = get_postgres_conn(True, True, 'postgres_database_1') +# conn2 = get_postgres_conn(True, True, 'postgres_database_2') +# +# cursor1 = conn1.cursor() +# cursor2 = conn2.cursor() +# +# create_clickhouse_postgres_db('postgres_database_1') +# create_clickhouse_postgres_db('postgres_database_2') +# +# cursors = [cursor1, cursor2] +# for cursor_id in range(len(cursors)): +# for i in range(NUM_TABLES): +# table_name = 'postgresql_replica_{}'.format(i) +# create_postgres_table(cursors[cursor_id], table_name); +# instance.query("INSERT INTO postgres_database_{}.{} SELECT number, number from numbers(50)".format(cursor_id + 1, table_name)) +# print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';''')) +# print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';''')) +# +# create_materialized_db('test_database_1', 'postgres_database_1') +# create_materialized_db('test_database_2', 'postgres_database_2') +# +# cursors = [cursor1, cursor2] +# for cursor_id in range(len(cursors)): +# for i in range(NUM_TABLES): +# table_name = 'postgresql_replica_{}'.format(i) +# instance.query("INSERT INTO postgres_database_{}.{} SELECT 50 + number, number from numbers(50)".format(cursor_id + 1, table_name)) +# +# for cursor_id in range(len(cursors)): +# for i in range(NUM_TABLES): +# table_name = 'postgresql_replica_{}'.format(i) +# check_tables_are_synchronized( +# table_name, 'key', 'postgres_database_{}'.format(cursor_id + 1), 'test_database_{}'.format(cursor_id + 1)); +# +# drop_clickhouse_postgres_db('postgres_database_1') +# drop_clickhouse_postgres_db('postgres_database_2') +# drop_materialized_db('test_database_1') +# drop_materialized_db('test_database_2') +# +# +#@pytest.mark.timeout(320) +#def test_concurrent_transactions(started_cluster): +# instance.query("DROP DATABASE IF EXISTS test_database") +# conn = get_postgres_conn(True) +# cursor = conn.cursor() +# NUM_TABLES = 6 +# +# for i in range(NUM_TABLES): +# create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); +# +# def transaction(thread_id): +# conn_ = get_postgres_conn(True, auto_commit=False) +# cursor_ = conn.cursor() +# for query in queries: +# cursor_.execute(query.format(thread_id)) +# print('thread {}, query {}'.format(thread_id, query)) +# conn_.commit() +# +# threads = [] +# threads_num = 6 +# for i in range(threads_num): +# threads.append(threading.Thread(target=transaction, args=(i,))) +# +# create_materialized_db() +# +# for thread in threads: +# time.sleep(random.uniform(0, 0.5)) +# thread.start() +# for thread in threads: +# thread.join() +# +# for i in range(NUM_TABLES): +# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); +# count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i)) +# count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i)) +# print(int(count1), int(count2), sep=' ') +# assert(int(count1) == int(count2)) +# drop_materialized_db() if __name__ == '__main__': diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 53eedbc8b7d..e448cfc8e99 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -16,21 +16,33 @@ postgres_table_template = """ key Integer NOT NULL, value Integer, PRIMARY KEY(key)) """ - -def get_postgres_conn(database=False): +def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database'): if database == True: - conn_string = "host='localhost' dbname='postgres_database' user='postgres' password='mysecretpassword'" + conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format(ip, port, database_name) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port) + conn = psycopg2.connect(conn_string) - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True + if auto_commit: + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True return conn def create_postgres_db(cursor, name): cursor.execute("CREATE DATABASE {}".format(name)) +def create_clickhouse_postgres_db(ip, port, name='postgres_database'): + instance.query(''' + CREATE DATABASE {} + ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')'''.format(name, ip, port, name)) + +def create_materialized_table(ip, port): + instance.query(''' + CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + ENGINE = MaterializePostgreSQL( + '{}:{}', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') + PRIMARY KEY key; '''.format(ip, port)) def create_postgres_table(cursor, table_name, replica_identity_full=False): cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) @@ -52,12 +64,13 @@ def postgresql_replica_check_result(result, check=False, ref_file='test_postgres def started_cluster(): try: cluster.start() - conn = get_postgres_conn() + conn = get_postgres_conn(ip=cluster.postgres_ip, + port=cluster.postgres_port) cursor = conn.cursor() create_postgres_db(cursor, 'postgres_database') - instance.query(''' - CREATE DATABASE postgres_database - ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')''') + create_clickhouse_postgres_db(ip=cluster.postgres_ip, + port=cluster.postgres_port) + instance.query('CREATE DATABASE test') yield cluster @@ -65,25 +78,23 @@ def started_cluster(): cluster.shutdown() @pytest.fixture(autouse=True) -def rabbitmq_setup_teardown(): +def postgresql_setup_teardown(): yield # run test instance.query('DROP TABLE IF EXISTS test.postgresql_replica') @pytest.mark.timeout(320) def test_initial_load_from_snapshot(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') while postgresql_replica_check_result(result) == False: @@ -96,18 +107,16 @@ def test_initial_load_from_snapshot(started_cluster): @pytest.mark.timeout(320) def test_no_connection_at_startup(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) time.sleep(3) instance.query('DETACH TABLE test.postgresql_replica') @@ -129,18 +138,16 @@ def test_no_connection_at_startup(started_cluster): @pytest.mark.timeout(320) def test_detach_attach_is_ok(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT count() FROM test.postgresql_replica;') while (int(result) == 0): @@ -164,19 +171,17 @@ def test_detach_attach_is_ok(started_cluster): @pytest.mark.timeout(320) def test_replicating_insert_queries(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT count() FROM test.postgresql_replica;') while (int(result) != 10): @@ -206,19 +211,17 @@ def test_replicating_insert_queries(started_cluster): @pytest.mark.timeout(320) def test_replicating_delete_queries(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;') while postgresql_replica_check_result(result) == False: @@ -245,19 +248,17 @@ def test_replicating_delete_queries(started_cluster): @pytest.mark.timeout(320) def test_replicating_update_queries(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT count() FROM test.postgresql_replica;') while (int(result) != 50): @@ -277,18 +278,16 @@ def test_replicating_update_queries(started_cluster): @pytest.mark.timeout(320) def test_resume_from_written_version(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT count() FROM test.postgresql_replica;') while (int(result) != 50): @@ -320,18 +319,16 @@ def test_resume_from_written_version(started_cluster): @pytest.mark.timeout(320) def test_many_replication_messages(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, PRIMARY KEY(key)) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - SETTINGS materialize_postgresql_max_block_size = 50000; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) result = instance.query('SELECT count() FROM test.postgresql_replica;') while (int(result) != 100000): @@ -375,18 +372,16 @@ def test_many_replication_messages(started_cluster): @pytest.mark.timeout(320) def test_connection_loss(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; - ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) i = 50 while i < 100000: @@ -412,17 +407,16 @@ def test_connection_loss(started_cluster): @pytest.mark.timeout(320) def test_clickhouse_restart(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)") instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) i = 50 while i < 100000: @@ -442,16 +436,15 @@ def test_clickhouse_restart(started_cluster): def test_rename_table(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)") @@ -477,16 +470,15 @@ def test_rename_table(started_cluster): def test_virtual_columns(started_cluster): - conn = get_postgres_conn(True) + 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'); instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - instance.query(''' - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( - 'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; ''') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)") result = instance.query('SELECT count() FROM test.postgresql_replica;') From 159de92197ec6426588763190679caa3ec4186d3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 26 Jun 2021 22:05:20 +0000 Subject: [PATCH 094/105] Uncomment test --- .../test.py | 174 +++++++++--------- 1 file changed, 87 insertions(+), 87 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index f19a5cf2467..685ed85d8f4 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -637,93 +637,93 @@ def test_virtual_columns(started_cluster): drop_materialized_db() -#def test_multiple_databases(started_cluster): -# instance.query("DROP DATABASE IF EXISTS test_database_1") -# instance.query("DROP DATABASE IF EXISTS test_database_2") -# NUM_TABLES = 5 -# -# conn = get_postgres_conn() -# cursor = conn.cursor() -# create_postgres_db(cursor, 'postgres_database_1') -# create_postgres_db(cursor, 'postgres_database_2') -# -# conn1 = get_postgres_conn(True, True, 'postgres_database_1') -# conn2 = get_postgres_conn(True, True, 'postgres_database_2') -# -# cursor1 = conn1.cursor() -# cursor2 = conn2.cursor() -# -# create_clickhouse_postgres_db('postgres_database_1') -# create_clickhouse_postgres_db('postgres_database_2') -# -# cursors = [cursor1, cursor2] -# for cursor_id in range(len(cursors)): -# for i in range(NUM_TABLES): -# table_name = 'postgresql_replica_{}'.format(i) -# create_postgres_table(cursors[cursor_id], table_name); -# instance.query("INSERT INTO postgres_database_{}.{} SELECT number, number from numbers(50)".format(cursor_id + 1, table_name)) -# print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';''')) -# print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';''')) -# -# create_materialized_db('test_database_1', 'postgres_database_1') -# create_materialized_db('test_database_2', 'postgres_database_2') -# -# cursors = [cursor1, cursor2] -# for cursor_id in range(len(cursors)): -# for i in range(NUM_TABLES): -# table_name = 'postgresql_replica_{}'.format(i) -# instance.query("INSERT INTO postgres_database_{}.{} SELECT 50 + number, number from numbers(50)".format(cursor_id + 1, table_name)) -# -# for cursor_id in range(len(cursors)): -# for i in range(NUM_TABLES): -# table_name = 'postgresql_replica_{}'.format(i) -# check_tables_are_synchronized( -# table_name, 'key', 'postgres_database_{}'.format(cursor_id + 1), 'test_database_{}'.format(cursor_id + 1)); -# -# drop_clickhouse_postgres_db('postgres_database_1') -# drop_clickhouse_postgres_db('postgres_database_2') -# drop_materialized_db('test_database_1') -# drop_materialized_db('test_database_2') -# -# -#@pytest.mark.timeout(320) -#def test_concurrent_transactions(started_cluster): -# instance.query("DROP DATABASE IF EXISTS test_database") -# conn = get_postgres_conn(True) -# cursor = conn.cursor() -# NUM_TABLES = 6 -# -# for i in range(NUM_TABLES): -# create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); -# -# def transaction(thread_id): -# conn_ = get_postgres_conn(True, auto_commit=False) -# cursor_ = conn.cursor() -# for query in queries: -# cursor_.execute(query.format(thread_id)) -# print('thread {}, query {}'.format(thread_id, query)) -# conn_.commit() -# -# threads = [] -# threads_num = 6 -# for i in range(threads_num): -# threads.append(threading.Thread(target=transaction, args=(i,))) -# -# create_materialized_db() -# -# for thread in threads: -# time.sleep(random.uniform(0, 0.5)) -# thread.start() -# for thread in threads: -# thread.join() -# -# for i in range(NUM_TABLES): -# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); -# count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i)) -# count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i)) -# print(int(count1), int(count2), sep=' ') -# assert(int(count1) == int(count2)) -# drop_materialized_db() +def test_multiple_databases(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database_1") + instance.query("DROP DATABASE IF EXISTS test_database_2") + NUM_TABLES = 5 + + conn = get_postgres_conn() + cursor = conn.cursor() + create_postgres_db(cursor, 'postgres_database_1') + create_postgres_db(cursor, 'postgres_database_2') + + conn1 = get_postgres_conn(True, True, 'postgres_database_1') + conn2 = get_postgres_conn(True, True, 'postgres_database_2') + + cursor1 = conn1.cursor() + cursor2 = conn2.cursor() + + create_clickhouse_postgres_db('postgres_database_1') + create_clickhouse_postgres_db('postgres_database_2') + + cursors = [cursor1, cursor2] + for cursor_id in range(len(cursors)): + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table(cursors[cursor_id], table_name); + instance.query("INSERT INTO postgres_database_{}.{} SELECT number, number from numbers(50)".format(cursor_id + 1, table_name)) + print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';''')) + print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';''')) + + create_materialized_db('test_database_1', 'postgres_database_1') + create_materialized_db('test_database_2', 'postgres_database_2') + + cursors = [cursor1, cursor2] + for cursor_id in range(len(cursors)): + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + instance.query("INSERT INTO postgres_database_{}.{} SELECT 50 + number, number from numbers(50)".format(cursor_id + 1, table_name)) + + for cursor_id in range(len(cursors)): + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + check_tables_are_synchronized( + table_name, 'key', 'postgres_database_{}'.format(cursor_id + 1), 'test_database_{}'.format(cursor_id + 1)); + + drop_clickhouse_postgres_db('postgres_database_1') + drop_clickhouse_postgres_db('postgres_database_2') + drop_materialized_db('test_database_1') + drop_materialized_db('test_database_2') + + +@pytest.mark.timeout(320) +def test_concurrent_transactions(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(True) + cursor = conn.cursor() + NUM_TABLES = 6 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + + def transaction(thread_id): + conn_ = get_postgres_conn(True, auto_commit=False) + cursor_ = conn.cursor() + for query in queries: + cursor_.execute(query.format(thread_id)) + print('thread {}, query {}'.format(thread_id, query)) + conn_.commit() + + threads = [] + threads_num = 6 + for i in range(threads_num): + threads.append(threading.Thread(target=transaction, args=(i,))) + + create_materialized_db() + + for thread in threads: + time.sleep(random.uniform(0, 0.5)) + thread.start() + for thread in threads: + thread.join() + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i)) + count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i)) + print(int(count1), int(count2), sep=' ') + assert(int(count1) == int(count2)) + drop_materialized_db() if __name__ == '__main__': From db998c3f6caa7f4f20442caeacc818bbfec4bacd Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 27 Jun 2021 16:15:28 +0000 Subject: [PATCH 095/105] More robust --- src/Core/PostgreSQL/Connection.cpp | 26 ++++++++- src/Core/PostgreSQL/Connection.h | 6 +- .../MaterializePostgreSQLConsumer.cpp | 58 +++++++++++++------ .../PostgreSQLReplicationHandler.cpp | 35 +++++++---- .../test.py | 40 ++++++++----- 5 files changed, 117 insertions(+), 48 deletions(-) diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp index ff6197d1390..71dfa7ea305 100644 --- a/src/Core/PostgreSQL/Connection.cpp +++ b/src/Core/PostgreSQL/Connection.cpp @@ -4,8 +4,8 @@ namespace postgres { -Connection::Connection(const ConnectionInfo & connection_info_, bool replication_) - : connection_info(connection_info_), replication(replication_) +Connection::Connection(const ConnectionInfo & connection_info_, bool replication_, size_t num_tries_) + : connection_info(connection_info_), replication(replication_), num_tries(num_tries_) { if (replication) { @@ -14,10 +14,30 @@ Connection::Connection(const ConnectionInfo & connection_info_, bool replication } } +void Connection::execWithRetry(const std::function & exec) +{ + for (size_t try_no = 0; try_no < num_tries; ++try_no) + { + try + { + pqxx::nontransaction tx(getRef()); + exec(tx); + } + catch (const pqxx::broken_connection & e) + { + LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaConnection"), + "Cannot execute query due to connection failure, attempt: {}/{}. (Message: {})", + try_no, num_tries, e.what()); + + if (try_no == num_tries) + throw; + } + } +} + pqxx::connection & Connection::getRef() { connect(); - assert(connection != nullptr); return *connection; } diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h index 46646ea6f35..00cf0c737f6 100644 --- a/src/Core/PostgreSQL/Connection.h +++ b/src/Core/PostgreSQL/Connection.h @@ -13,7 +13,9 @@ using ConnectionPtr = std::unique_ptr; class Connection : private boost::noncopyable { public: - Connection(const ConnectionInfo & connection_info_, bool replication_ = false); + Connection(const ConnectionInfo & connection_info_, bool replication_ = false, size_t num_tries = 3); + + void execWithRetry(const std::function & exec); pqxx::connection & getRef(); @@ -24,6 +26,8 @@ public: private: ConnectionPtr connection; ConnectionInfo connection_info; + bool replication; + size_t num_tries; }; } diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp index b693dad6a68..b1325d9ca57 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp @@ -252,7 +252,7 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli /// Skip '\x' size_t pos = 2; char type = readInt8(replication_message, pos, size); - //LOG_DEBUG(log, "Message type: {}, lsn string: {}, lsn value {}", type, current_lsn, lsn_value); + // LOG_DEBUG(log, "Message type: {}, lsn string: {}, lsn value {}", type, current_lsn, lsn_value); switch (type) { @@ -352,9 +352,9 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli constexpr size_t transaction_commit_timestamp_len = 8; pos += unused_flags_len + commit_lsn_len + transaction_end_lsn_len + transaction_commit_timestamp_len; - final_lsn = current_lsn; - LOG_DEBUG(log, "Commit lsn: {}", getLSNValue(current_lsn)); /// Will be removed + LOG_DEBUG(log, "Current lsn: {} = {}", current_lsn, getLSNValue(current_lsn)); /// Will be removed + final_lsn = current_lsn; break; } case 'R': // Relation @@ -458,9 +458,9 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli void MaterializePostgreSQLConsumer::syncTables(std::shared_ptr tx) { - for (const auto & table_name : tables_to_sync) + try { - try + for (const auto & table_name : tables_to_sync) { auto & buffer = buffers.find(table_name)->second; Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns)); @@ -483,19 +483,20 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptrgetHeader(), "postgresql replica table sync"); copyData(input, *block_io.out); - current_lsn = advanceLSN(tx); buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); } } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - LOG_DEBUG(log, "Table sync end for {} tables", tables_to_sync.size()); - tables_to_sync.clear(); - tx->commit(); + LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); + + current_lsn = advanceLSN(tx); + tables_to_sync.clear(); + tx->commit(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } @@ -507,6 +508,7 @@ String MaterializePostgreSQLConsumer::advanceLSN(std::shared_ptr(); + LOG_TRACE(log, "Advanced LSN up to: {}", final_lsn); return final_lsn; } @@ -622,14 +624,34 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() return false; } - catch (const Exception & e) + catch (const pqxx::conversion_error & e) + { + LOG_ERROR(log, "Convertion error: {}", e.what()); + return false; + } + catch (const pqxx::broken_connection & e) + { + LOG_ERROR(log, "Connection error: {}", e.what()); + return false; + } + catch (const Exception &) { - if (e.code() == ErrorCodes::UNKNOWN_TABLE) - throw; - tryLogCurrentException(__PRETTY_FUNCTION__); return false; } + catch (...) + { + /// Since reading is done from a background task, it is important to catch any possible error + /// in order to understand why something does not work. + try + { + std::rethrow_exception(std::current_exception()); + } + catch (const std::exception& e) + { + LOG_ERROR(log, "Unexpected error: {}", e.what()); + } + } if (!tables_to_sync.empty()) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 33d5c49ec09..46fedb99b62 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -15,7 +15,7 @@ namespace DB { static const auto RESCHEDULE_MS = 500; -static const auto BACKOFF_TRESHOLD = 32000; +static const auto BACKOFF_TRESHOLD = 5000; namespace ErrorCodes { @@ -255,18 +255,25 @@ void PostgreSQLReplicationHandler::consumerFunc() } if (stop_synchronization) + { + LOG_TRACE(log, "Replication thread is stopped"); return; + } if (schedule_now) { - consumer_task->schedule(); milliseconds_to_wait = RESCHEDULE_MS; + consumer_task->schedule(); + + LOG_DEBUG(log, "Scheduling replication thread: now"); } else { consumer_task->scheduleAfter(milliseconds_to_wait); if (milliseconds_to_wait < BACKOFF_TRESHOLD) milliseconds_to_wait *= 2; + + LOG_TRACE(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); } } @@ -397,16 +404,24 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::shutdownFinal() { - pqxx::nontransaction tx(connection->getRef()); - dropPublication(tx); - String last_committed_lsn; try { - if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */false)) - dropReplicationSlot(tx, /* temporary */false); - if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true)) - dropReplicationSlot(tx, /* temporary */true); - tx.commit(); + shutdown(); + + connection->execWithRetry([&](pqxx::nontransaction & tx){ dropPublication(tx); }); + String last_committed_lsn; + + connection->execWithRetry([&](pqxx::nontransaction & tx) + { + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */false)) + dropReplicationSlot(tx, /* temporary */false); + }); + + connection->execWithRetry([&](pqxx::nontransaction & tx) + { + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true)) + dropReplicationSlot(tx, /* temporary */true); + }); } catch (Exception & e) { diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 685ed85d8f4..d1e590704fd 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -145,11 +145,6 @@ def started_cluster(): cluster.shutdown() -@pytest.fixture(autouse=True) -def postgresql_setup_teardown(): - yield # run test - - @pytest.mark.timeout(120) def test_load_and_sync_all_database_tables(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") @@ -642,19 +637,25 @@ def test_multiple_databases(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database_2") NUM_TABLES = 5 - conn = get_postgres_conn() + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=False) cursor = conn.cursor() create_postgres_db(cursor, 'postgres_database_1') create_postgres_db(cursor, 'postgres_database_2') - conn1 = get_postgres_conn(True, True, 'postgres_database_1') - conn2 = get_postgres_conn(True, True, 'postgres_database_2') + conn1 = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, database_name='postgres_database_1') + conn2 = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, database_name='postgres_database_2') cursor1 = conn1.cursor() cursor2 = conn2.cursor() - create_clickhouse_postgres_db('postgres_database_1') - create_clickhouse_postgres_db('postgres_database_2') + create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_1') + create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_2') cursors = [cursor1, cursor2] for cursor_id in range(len(cursors)): @@ -665,8 +666,10 @@ def test_multiple_databases(started_cluster): print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';''')) print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';''')) - create_materialized_db('test_database_1', 'postgres_database_1') - create_materialized_db('test_database_2', 'postgres_database_2') + create_materialized_db(started_cluster.postgres_ip, started_cluster.postgres_port, + 'test_database_1', 'postgres_database_1') + create_materialized_db(started_cluster.postgres_ip, started_cluster.postgres_port, + 'test_database_2', 'postgres_database_2') cursors = [cursor1, cursor2] for cursor_id in range(len(cursors)): @@ -689,7 +692,9 @@ def test_multiple_databases(started_cluster): @pytest.mark.timeout(320) def test_concurrent_transactions(started_cluster): instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn(True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) cursor = conn.cursor() NUM_TABLES = 6 @@ -697,19 +702,22 @@ def test_concurrent_transactions(started_cluster): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); def transaction(thread_id): - conn_ = get_postgres_conn(True, auto_commit=False) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=False) cursor_ = conn.cursor() for query in queries: cursor_.execute(query.format(thread_id)) print('thread {}, query {}'.format(thread_id, query)) - conn_.commit() + conn.commit() threads = [] threads_num = 6 for i in range(threads_num): threads.append(threading.Thread(target=transaction, args=(i,))) - create_materialized_db() + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) for thread in threads: time.sleep(random.uniform(0, 0.5)) From c3a2fc05846255a8b947c806b334930fbb6af42e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 27 Jun 2021 19:09:17 +0000 Subject: [PATCH 096/105] Review fixes --- ...stgresql.md => materialized-postgresql.md} | 23 +-- ...stgresql.md => materialized-postgresql.md} | 13 +- src/Core/Settings.h | 2 +- src/Databases/DatabaseFactory.cpp | 14 +- ...cpp => DatabaseMaterializedPostgreSQL.cpp} | 58 +++---- ...SQL.h => DatabaseMaterializedPostgreSQL.h} | 12 +- src/Interpreters/InterpreterCreateQuery.cpp | 6 +- src/Interpreters/InterpreterDropQuery.cpp | 4 +- .../MaterializePostgreSQLSettings.h | 30 ---- ...cpp => MaterializedPostgreSQLConsumer.cpp} | 47 +++--- ...mer.h => MaterializedPostgreSQLConsumer.h} | 4 +- ...cpp => MaterializedPostgreSQLSettings.cpp} | 6 +- .../MaterializedPostgreSQLSettings.h | 30 ++++ .../PostgreSQLReplicationHandler.cpp | 153 ++++++++++-------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 20 +-- ....cpp => StorageMaterializedPostgreSQL.cpp} | 117 ++++++++------ ...eSQL.h => StorageMaterializedPostgreSQL.h} | 55 ++++--- src/Storages/registerStorages.cpp | 4 +- .../configs/users.xml | 2 +- .../test.py | 24 +-- .../test_storage_postgresql_replica/test.py | 2 +- 21 files changed, 330 insertions(+), 296 deletions(-) rename docs/en/engines/database-engines/{materialize-postgresql.md => materialized-postgresql.md} (59%) rename docs/en/engines/table-engines/integrations/{materialize-postgresql.md => materialized-postgresql.md} (54%) rename src/Databases/PostgreSQL/{DatabaseMaterializePostgreSQL.cpp => DatabaseMaterializedPostgreSQL.cpp} (66%) rename src/Databases/PostgreSQL/{DatabaseMaterializePostgreSQL.h => DatabaseMaterializedPostgreSQL.h} (83%) delete mode 100644 src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h rename src/Storages/PostgreSQL/{MaterializePostgreSQLConsumer.cpp => MaterializedPostgreSQLConsumer.cpp} (91%) rename src/Storages/PostgreSQL/{MaterializePostgreSQLConsumer.h => MaterializedPostgreSQLConsumer.h} (98%) rename src/Storages/PostgreSQL/{MaterializePostgreSQLSettings.cpp => MaterializedPostgreSQLSettings.cpp} (77%) create mode 100644 src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h rename src/Storages/PostgreSQL/{StorageMaterializePostgreSQL.cpp => StorageMaterializedPostgreSQL.cpp} (78%) rename src/Storages/PostgreSQL/{StorageMaterializePostgreSQL.h => StorageMaterializedPostgreSQL.h} (73%) diff --git a/docs/en/engines/database-engines/materialize-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md similarity index 59% rename from docs/en/engines/database-engines/materialize-postgresql.md rename to docs/en/engines/database-engines/materialized-postgresql.md index f657035d050..dde3886f694 100644 --- a/docs/en/engines/database-engines/materialize-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -1,15 +1,15 @@ --- toc_priority: 30 -toc_title: MaterializePostgreSQL +toc_title: MaterializedPostgreSQL --- -# MaterializePostgreSQL {#materialize-postgresql} +# MaterializedPostgreSQL {#materialize-postgresql} ## Creating a Database {#creating-a-database} ``` sql CREATE DATABASE test_database -ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' SELECT * FROM test_database.postgres_table; ``` @@ -17,17 +17,17 @@ SELECT * FROM test_database.postgres_table; ## Settings {#settings} -1. `materialize_postgresql_max_block_size` - Number of rows collected before flushing data into table. Default: `65536`. +1. `materialized_postgresql_max_block_size` - Number of rows collected before flushing data into table. Default: `65536`. -2. `materialize_postgresql_tables_list` - List of tables for MaterializePostgreSQL database engine. Default: `whole database`. +2. `materialized_postgresql_tables_list` - List of tables for MaterializedPostgreSQL database engine. Default: `whole database`. -3. `materialize_postgresql_allow_automatic_update` - Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). +3. `materialized_postgresql_allow_automatic_update` - Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`). ``` sql CREATE DATABASE test_database -ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' -SETTINGS materialize_postgresql_max_block_size = 65536, - materialize_postgresql_tables_list = 'table1,table2,table3'; +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password' +SETTINGS materialized_postgresql_max_block_size = 65536, + materialized_postgresql_tables_list = 'table1,table2,table3'; SELECT * FROM test_database.table1; ``` @@ -64,3 +64,8 @@ postgres# SELECT CASE relreplident FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` + + +## WARNINGS {#warnings} + +1. **TOAST** values convertions is not supported. Default value for the data type will be used. diff --git a/docs/en/engines/table-engines/integrations/materialize-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md similarity index 54% rename from docs/en/engines/table-engines/integrations/materialize-postgresql.md rename to docs/en/engines/table-engines/integrations/materialized-postgresql.md index aba1a370792..8c645d03e13 100644 --- a/docs/en/engines/table-engines/integrations/materialize-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -3,13 +3,13 @@ toc_priority: 12 toc_title: MateriaziePostgreSQL --- -# MaterializePostgreSQL {#materialize-postgresql} +# MaterializedPostgreSQL {#materialize-postgresql} ## Creating a Table {#creating-a-table} ``` sql CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) -ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') PRIMARY KEY key; ``` @@ -18,7 +18,7 @@ PRIMARY KEY key; - Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. -- A table with engine `MaterializePostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialize-postgresql.md#requirements)). +- A table with engine `MaterializedPostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialize-postgresql.md#requirements)). - Only database `Atomic` is allowed. @@ -34,8 +34,13 @@ These columns do not need to be added, when table is created. They are always ac ``` sql CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) -ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password') PRIMARY KEY key; SELECT key, value, _version FROM test.postgresql_replica; ``` + + +## WARNINGS {#warnings} + +1. **TOAST** values convertions is not supported. Default value for the data type will be used. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 69b9e8bc10f..2bf5aeaeed3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -428,7 +428,7 @@ class IColumn; M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ - M(Bool, allow_experimental_database_materialize_postgresql, false, "Allow to create database with Engine=MaterializePostgreSQL(...).", 0) \ + M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index adfefcf7820..802d50d11c2 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -36,8 +36,8 @@ #if USE_LIBPQXX #include // Y_IGNORE -#include -#include +#include +#include #endif namespace fs = std::filesystem; @@ -100,14 +100,14 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const UUID & uuid = create.uuid; bool engine_may_have_arguments = engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "Lazy" || - engine_name == "Replicated" || engine_name == "PostgreSQL" || engine_name == "MaterializePostgreSQL"; + engine_name == "Replicated" || engine_name == "PostgreSQL" || engine_name == "MaterializedPostgreSQL"; if (engine_define->engine->arguments && !engine_may_have_arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || engine_define->sample_by; - bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializePostgreSQL"; + bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL"; if (has_unexpected_element || (!may_have_settings && engine_define->settings)) throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); @@ -263,7 +263,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared( context, metadata_path, engine_define, database_name, postgres_database_name, connection_pool, use_table_cache); } - else if (engine_name == "MaterializePostgreSQL") + else if (engine_name == "MaterializedPostgreSQL") { const ASTFunction * engine = engine_define->engine; @@ -287,12 +287,12 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String auto parsed_host_port = parseAddress(host_port, 5432); auto connection_info = postgres::formatConnectionString(postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password); - auto postgresql_replica_settings = std::make_unique(); + auto postgresql_replica_settings = std::make_unique(); if (engine_define->settings) postgresql_replica_settings->loadFromQuery(*engine_define); - return std::make_shared( + return std::make_shared( context, metadata_path, uuid, engine_define, database_name, postgres_database_name, connection_info, std::move(postgresql_replica_settings)); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp similarity index 66% rename from src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp rename to src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index f3d71d6bf67..37a464c6cda 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -1,8 +1,8 @@ -#include +#include #if USE_LIBPQXX -#include +#include #include #include @@ -31,7 +31,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( +DatabaseMaterializedPostgreSQL::DatabaseMaterializedPostgreSQL( ContextPtr context_, const String & metadata_path_, UUID uuid_, @@ -39,8 +39,8 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( const String & database_name_, const String & postgres_database_name, const postgres::ConnectionInfo & connection_info_, - std::unique_ptr settings_) - : DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializePostgreSQL (" + database_name_ + ")", context_) + std::unique_ptr settings_) + : DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializedPostgreSQL (" + database_name_ + ")", context_) , database_engine_define(database_engine_define_->clone()) , remote_database_name(postgres_database_name) , connection_info(connection_info_) @@ -49,7 +49,7 @@ DatabaseMaterializePostgreSQL::DatabaseMaterializePostgreSQL( } -void DatabaseMaterializePostgreSQL::startSynchronization() +void DatabaseMaterializedPostgreSQL::startSynchronization() { replication_handler = std::make_unique( /* replication_identifier */database_name, @@ -57,10 +57,10 @@ void DatabaseMaterializePostgreSQL::startSynchronization() database_name, connection_info, getContext(), - settings->materialize_postgresql_max_block_size.value, - settings->materialize_postgresql_allow_automatic_update, - /* is_materialize_postgresql_database = */ true, - settings->materialize_postgresql_tables_list.value); + settings->materialized_postgresql_max_block_size.value, + settings->materialized_postgresql_allow_automatic_update, + /* is_materialized_postgresql_database = */ true, + settings->materialized_postgresql_tables_list.value); postgres::Connection connection(connection_info); std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection.getRef()); @@ -73,19 +73,19 @@ void DatabaseMaterializePostgreSQL::startSynchronization() if (storage) { /// Nested table was already created and synchronized. - storage = StorageMaterializePostgreSQL::create(storage, getContext()); + storage = StorageMaterializedPostgreSQL::create(storage, getContext()); } else { /// Nested table does not exist and will be created by replication thread. - storage = StorageMaterializePostgreSQL::create(StorageID(database_name, table_name), getContext()); + storage = StorageMaterializedPostgreSQL::create(StorageID(database_name, table_name), getContext()); } - /// Cache MaterializePostgreSQL wrapper over nested table. + /// Cache MaterializedPostgreSQL wrapper over nested table. materialized_tables[table_name] = storage; - /// Let replication thread now, which tables it needs to keep in sync. - replication_handler->addStorage(table_name, storage->as()); + /// Let replication thread know, which tables it needs to keep in sync. + replication_handler->addStorage(table_name, storage->as()); } LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); @@ -93,7 +93,7 @@ void DatabaseMaterializePostgreSQL::startSynchronization() } -void DatabaseMaterializePostgreSQL::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseMaterializedPostgreSQL::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach) { DatabaseAtomic::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); @@ -112,9 +112,9 @@ void DatabaseMaterializePostgreSQL::loadStoredObjects(ContextMutablePtr local_co } -StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, ContextPtr local_context) const +StoragePtr DatabaseMaterializedPostgreSQL::tryGetTable(const String & name, ContextPtr local_context) const { - /// In otder to define which table access is needed - to MaterializePostgreSQL table (only in case of SELECT queries) or + /// In otder to define which table access is needed - to MaterializedPostgreSQL table (only in case of SELECT queries) or /// to its nested ReplacingMergeTree table (in all other cases), the context of a query os modified. /// Also if materialzied_tables set is empty - it means all access is done to ReplacingMergeTree tables - it is a case after /// replication_handler was shutdown. @@ -123,14 +123,14 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte return DatabaseAtomic::tryGetTable(name, local_context); } - /// Note: In select query we call MaterializePostgreSQL table and it calls tryGetTable from its nested. - /// So the only point, where synchronization is needed - access to MaterializePostgreSQL table wrapper over nested table. + /// Note: In select query we call MaterializedPostgreSQL table and it calls tryGetTable from its nested. + /// So the only point, where synchronization is needed - access to MaterializedPostgreSQL table wrapper over nested table. std::lock_guard lock(tables_mutex); auto table = materialized_tables.find(name); /// Return wrapper over ReplacingMergeTree table. If table synchronization just started, table will not /// be accessible immediately. Table is considered to exist once its nested table was created. - if (table != materialized_tables.end() && table->second->as ()->hasNested()) + if (table != materialized_tables.end() && table->second->as ()->hasNested()) { return table->second; } @@ -139,7 +139,7 @@ StoragePtr DatabaseMaterializePostgreSQL::tryGetTable(const String & name, Conte } -void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) { /// Create table query can only be called from replication thread. if (local_context->isInternalQuery()) @@ -153,7 +153,7 @@ void DatabaseMaterializePostgreSQL::createTable(ContextPtr local_context, const } -void DatabaseMaterializePostgreSQL::stopReplication() +void DatabaseMaterializedPostgreSQL::stopReplication() { if (replication_handler) replication_handler->shutdown(); @@ -163,27 +163,27 @@ void DatabaseMaterializePostgreSQL::stopReplication() } -void DatabaseMaterializePostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) +void DatabaseMaterializedPostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) { /// Modify context into nested_context and pass query to Atomic database. - DatabaseAtomic::dropTable(StorageMaterializePostgreSQL::makeNestedTableContext(local_context), table_name, no_delay); + DatabaseAtomic::dropTable(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), table_name, no_delay); } -void DatabaseMaterializePostgreSQL::drop(ContextPtr local_context) +void DatabaseMaterializedPostgreSQL::drop(ContextPtr local_context) { if (replication_handler) replication_handler->shutdownFinal(); - DatabaseAtomic::drop(StorageMaterializePostgreSQL::makeNestedTableContext(local_context)); + DatabaseAtomic::drop(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context)); } -DatabaseTablesIteratorPtr DatabaseMaterializePostgreSQL::getTablesIterator( +DatabaseTablesIteratorPtr DatabaseMaterializedPostgreSQL::getTablesIterator( ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) { /// Modify context into nested_context and pass query to Atomic database. - return DatabaseAtomic::getTablesIterator(StorageMaterializePostgreSQL::makeNestedTableContext(local_context), filter_by_table_name); + return DatabaseAtomic::getTablesIterator(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), filter_by_table_name); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h similarity index 83% rename from src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h rename to src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index aff2db4499a..0a60f47cbe4 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -7,7 +7,7 @@ #if USE_LIBPQXX #include -#include +#include #include #include @@ -24,11 +24,11 @@ class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; -class DatabaseMaterializePostgreSQL : public DatabaseAtomic +class DatabaseMaterializedPostgreSQL : public DatabaseAtomic { public: - DatabaseMaterializePostgreSQL( + DatabaseMaterializedPostgreSQL( ContextPtr context_, const String & metadata_path_, UUID uuid_, @@ -36,9 +36,9 @@ public: const String & database_name_, const String & postgres_database_name, const postgres::ConnectionInfo & connection_info, - std::unique_ptr settings_); + std::unique_ptr settings_); - String getEngineName() const override { return "MaterializePostgreSQL"; } + String getEngineName() const override { return "MaterializedPostgreSQL"; } String getMetadataPath() const override { return metadata_path; } @@ -63,7 +63,7 @@ private: ASTPtr database_engine_define; String remote_database_name; postgres::ConnectionInfo connection_info; - std::unique_ptr settings; + std::unique_ptr settings; std::shared_ptr replication_handler; std::map materialized_tables; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 33c80ccee38..7ebc7b95ea9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -151,7 +151,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage)); } - if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated" || create.storage->engine->name == "MaterializePostgreSQL") + if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated" || create.storage->engine->name == "MaterializedPostgreSQL") { if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. " @@ -217,9 +217,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "MaterializePostgreSQL" && !getContext()->getSettingsRef().allow_experimental_database_materialize_postgresql && !internal) + if (create.storage->engine->name == "MaterializedPostgreSQL" && !getContext()->getSettingsRef().allow_experimental_database_materialized_postgresql && !internal) { - throw Exception("MaterializePostgreSQL is an experimental database engine. " + throw Exception("MaterializedPostgreSQL is an experimental database engine. " "Enable allow_experimental_database_postgresql_replica to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 9e4fb44f9a2..94d5fbf3ea7 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -21,7 +21,7 @@ #endif #if USE_LIBPQXX -# include +# include #endif namespace DB @@ -321,7 +321,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (auto * replicated = typeid_cast(database.get())) replicated->stopReplication(); #if USE_LIBPQXX - if (auto * materialize_postgresql = typeid_cast(database.get())) + if (auto * materialize_postgresql = typeid_cast(database.get())) materialize_postgresql->stopReplication(); #endif diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h deleted file mode 100644 index 8875c45f9fa..00000000000 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#if !defined(ARCADIA_BUILD) -#include "config_core.h" -#endif - -#if USE_LIBPQXX -#include - - -namespace DB -{ - class ASTStorage; - - -#define LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS(M) \ - M(UInt64, materialize_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \ - M(String, materialize_postgresql_tables_list, "", "List of tables for MaterializePostgreSQL database engine", 0) \ - M(Bool, materialize_postgresql_allow_automatic_update, 0, "Allow to reload table in the background, when schema changes are detected", 0) \ - -DECLARE_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) - -struct MaterializePostgreSQLSettings : public BaseSettings -{ - void loadFromQuery(ASTStorage & storage_def); -}; - -} - -#endif diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp similarity index 91% rename from src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp rename to src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index b1325d9ca57..390ad996e4e 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -1,6 +1,6 @@ -#include "MaterializePostgreSQLConsumer.h" +#include "MaterializedPostgreSQLConsumer.h" -#include "StorageMaterializePostgreSQL.h" +#include "StorageMaterializedPostgreSQL.h" #include #include #include @@ -16,10 +16,9 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int UNKNOWN_TABLE; } -MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( +MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( ContextPtr context_, std::shared_ptr connection_, const std::string & replication_slot_name_, @@ -46,7 +45,7 @@ MaterializePostgreSQLConsumer::MaterializePostgreSQLConsumer( } -void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage) +void MaterializedPostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage) { const auto storage_metadata = storage->getInMemoryMetadataPtr(); const Block sample_block = storage_metadata->getSampleBlock(); @@ -60,7 +59,7 @@ void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage auto insert_columns = std::make_shared(); auto table_id = storage->getStorageID(); - LOG_TRACE(&Poco::Logger::get("MaterializePostgreSQLBuffer"), "New buffer for table {}.{} ({}), structure: {}", + LOG_TRACE(&Poco::Logger::get("MaterializedPostgreSQLBuffer"), "New buffer for table {}.{} ({}), structure: {}", table_id.database_name, table_id.table_name, toString(table_id.uuid), sample_block.dumpStructure()); assert(description.sample_block.columns() == storage_columns.size()); @@ -79,7 +78,7 @@ void MaterializePostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage } -void MaterializePostgreSQLConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx) +void MaterializedPostgreSQLConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); bool is_nullable = buffer.description.types[column_idx].second; @@ -105,14 +104,14 @@ void MaterializePostgreSQLConsumer::insertValue(Buffer & buffer, const std::stri } -void MaterializePostgreSQLConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx) +void MaterializedPostgreSQLConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx) { const auto & sample = buffer.description.sample_block.getByPosition(column_idx); insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column); } -void MaterializePostgreSQLConsumer::readString(const char * message, size_t & pos, size_t size, String & result) +void MaterializedPostgreSQLConsumer::readString(const char * message, size_t & pos, size_t size, String & result) { assert(size > pos + 2); char current = unhex2(message + pos); @@ -127,7 +126,7 @@ void MaterializePostgreSQLConsumer::readString(const char * message, size_t & po template -T MaterializePostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t n) +T MaterializedPostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t n) { T result = 0; for (size_t i = 0; i < n; ++i) @@ -139,7 +138,7 @@ T MaterializePostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t } -Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int64 MaterializedPostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 16); Int64 result = unhexN(message, pos, 8); @@ -148,7 +147,7 @@ Int64 MaterializePostgreSQLConsumer::readInt64(const char * message, size_t & po } -Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int32 MaterializedPostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 8); Int32 result = unhexN(message, pos, 4); @@ -157,7 +156,7 @@ Int32 MaterializePostgreSQLConsumer::readInt32(const char * message, size_t & po } -Int16 MaterializePostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int16 MaterializedPostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 4); Int16 result = unhexN(message, pos, 2); @@ -166,7 +165,7 @@ Int16 MaterializePostgreSQLConsumer::readInt16(const char * message, size_t & po } -Int8 MaterializePostgreSQLConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) +Int8 MaterializedPostgreSQLConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 2); Int8 result = unhex2(message + pos); @@ -175,7 +174,7 @@ Int8 MaterializePostgreSQLConsumer::readInt8(const char * message, size_t & pos, } -void MaterializePostgreSQLConsumer::readTupleData( +void MaterializedPostgreSQLConsumer::readTupleData( Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) { Int16 num_columns = readInt16(message, pos, size); @@ -247,7 +246,7 @@ void MaterializePostgreSQLConsumer::readTupleData( /// https://www.postgresql.org/docs/13/protocol-logicalrep-message-formats.html -void MaterializePostgreSQLConsumer::processReplicationMessage(const char * replication_message, size_t size) +void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * replication_message, size_t size) { /// Skip '\x' size_t pos = 2; @@ -456,7 +455,7 @@ void MaterializePostgreSQLConsumer::processReplicationMessage(const char * repli } -void MaterializePostgreSQLConsumer::syncTables(std::shared_ptr tx) +void MaterializedPostgreSQLConsumer::syncTables(std::shared_ptr tx) { try { @@ -500,7 +499,7 @@ void MaterializePostgreSQLConsumer::syncTables(std::shared_ptr tx) +String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptr tx) { std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn); pqxx::result result{tx->exec(query_str)}; @@ -516,7 +515,7 @@ String MaterializePostgreSQLConsumer::advanceLSN(std::shared_ptr tx; bool slot_empty = true; @@ -626,7 +625,7 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() } catch (const pqxx::conversion_error & e) { - LOG_ERROR(log, "Convertion error: {}", e.what()); + LOG_ERROR(log, "Conversion error: {}", e.what()); return false; } catch (const pqxx::broken_connection & e) @@ -662,7 +661,7 @@ bool MaterializePostgreSQLConsumer::readFromReplicationSlot() } -bool MaterializePostgreSQLConsumer::consume(std::vector> & skipped_tables) +bool MaterializedPostgreSQLConsumer::consume(std::vector> & skipped_tables) { /// Check if there are tables, which are skipped from being updated by changes from replication stream, /// because schema changes were detected. Update them, if it is allowed. @@ -687,7 +686,7 @@ bool MaterializePostgreSQLConsumer::consume(std::vector } -void MaterializePostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn) +void MaterializedPostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn) { /// Cache new pointer to replacingMergeTree table. storages[table_name] = nested_storage; diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h similarity index 98% rename from src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h rename to src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 00523ff0ea9..59feb60a21a 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -14,12 +14,12 @@ namespace DB { -class MaterializePostgreSQLConsumer +class MaterializedPostgreSQLConsumer { public: using Storages = std::unordered_map; - MaterializePostgreSQLConsumer( + MaterializedPostgreSQLConsumer( ContextPtr context_, std::shared_ptr connection_, const String & replication_slot_name_, diff --git a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp similarity index 77% rename from src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp rename to src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp index 2682bd6194f..ef9ca78d984 100644 --- a/src/Storages/PostgreSQL/MaterializePostgreSQLSettings.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp @@ -1,4 +1,4 @@ -#include "MaterializePostgreSQLSettings.h" +#include "MaterializedPostgreSQLSettings.h" #if USE_LIBPQXX #include @@ -15,9 +15,9 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_TRAITS(MaterializePostgreSQLSettingsTraits, LIST_OF_MATERIALIZE_POSTGRESQL_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) -void MaterializePostgreSQLSettings::loadFromQuery(ASTStorage & storage_def) +void MaterializedPostgreSQLSettings::loadFromQuery(ASTStorage & storage_def) { if (storage_def.settings) { diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h new file mode 100644 index 00000000000..3bc32a21876 --- /dev/null +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -0,0 +1,30 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX +#include + + +namespace DB +{ + class ASTStorage; + + +#define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M) \ + M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \ + M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \ + M(Bool, materialized_postgresql_allow_automatic_update, 0, "Allow to reload table in the background, when schema changes are detected", 0) \ + +DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) + +struct MaterializedPostgreSQLSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} + +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 46fedb99b62..74e3a2fb965 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -15,7 +15,7 @@ namespace DB { static const auto RESCHEDULE_MS = 500; -static const auto BACKOFF_TRESHOLD = 5000; +static const auto BACKOFF_TRESHOLD_MS = 10000; namespace ErrorCodes { @@ -30,7 +30,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( ContextPtr context_, const size_t max_block_size_, bool allow_automatic_update_, - bool is_materialize_postgresql_database_, + bool is_materialized_postgresql_database_, const String tables_list_) : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , context(context_) @@ -39,7 +39,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , connection_info(connection_info_) , max_block_size(max_block_size_) , allow_automatic_update(allow_automatic_update_) - , is_materialize_postgresql_database(is_materialize_postgresql_database_) + , is_materialized_postgresql_database(is_materialized_postgresql_database_) , tables_list(tables_list_) , connection(std::make_shared(connection_info_)) , milliseconds_to_wait(RESCHEDULE_MS) @@ -52,7 +52,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } -void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage) +void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StorageMaterializedPostgreSQL * storage) { materialized_storages[table_name] = storage; } @@ -122,14 +122,14 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) { try { - nested_storages[table_name] = loadFromSnapshot(snapshot_name, table_name, storage->as ()); + nested_storages[table_name] = loadFromSnapshot(snapshot_name, table_name, storage->as ()); } catch (Exception & e) { e.addMessage("while loading table {}.{}", remote_database_name, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); - /// Throw in case of single MaterializePostgreSQL storage, because initial setup is done immediately + /// Throw in case of single MaterializedPostgreSQL storage, because initial setup is done immediately /// (unlike database engine where it is done in a separate thread). if (throw_on_error) throw; @@ -137,7 +137,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } }; - /// There is one replication slot for each replication handler. In case of MaterializePostgreSQL database engine, + /// There is one replication slot for each replication handler. In case of MaterializedPostgreSQL database engine, /// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler. /// Recreation of a replication slot imposes reloading of all tables. if (!isReplicationSlotExist(tx, start_lsn, /* temporary */false)) @@ -159,7 +159,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) LOG_TRACE(log, "Loading {} tables...", materialized_storages.size()); for (const auto & [table_name, storage] : materialized_storages) { - auto * materialized_storage = storage->as (); + auto * materialized_storage = storage->as (); try { /// Try load nested table, set materialized table metadata. @@ -181,7 +181,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// Pass current connection to consumer. It is not std::moved implicitly, but a shared_ptr is passed. /// Consumer and replication handler are always executed one after another (not concurrently) and share the same connection. /// Handler uses it only for loadFromSnapshot and shutdown methods. - consumer = std::make_shared( + consumer = std::make_shared( context, connection, replication_slot, @@ -199,7 +199,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name, const String & table_name, - StorageMaterializePostgreSQL * materialized_storage) + StorageMaterializedPostgreSQL * materialized_storage) { auto tx = std::make_shared(connection->getRef()); @@ -270,7 +270,7 @@ void PostgreSQLReplicationHandler::consumerFunc() else { consumer_task->scheduleAfter(milliseconds_to_wait); - if (milliseconds_to_wait < BACKOFF_TRESHOLD) + if (milliseconds_to_wait < BACKOFF_TRESHOLD_MS) milliseconds_to_wait *= 2; LOG_TRACE(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); @@ -432,7 +432,7 @@ void PostgreSQLReplicationHandler::shutdownFinal() } -/// Used by MaterializePostgreSQL database engine. +/// Used by MaterializedPostgreSQL database engine. NameSet PostgreSQLReplicationHandler::fetchRequiredTables(pqxx::connection & connection_) { pqxx::work tx(connection_); @@ -474,7 +474,7 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( pqxx::ReplicationTransaction & tx, const std::string & table_name) const { - if (!is_materialize_postgresql_database) + if (!is_materialized_postgresql_database) return nullptr; return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, true, true, true)); @@ -486,81 +486,92 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras (); + postgres::Connection replication_connection(connection_info, /* replication */true); + pqxx::nontransaction tx(replication_connection.getRef()); - auto temp_materialized_storage = materialized_storage->createTemporary(); + String snapshot_name, start_lsn; - /// This snapshot is valid up to the end of the transaction, which exported it. - StoragePtr temp_nested_storage = loadFromSnapshot(snapshot_name, table_name, temp_materialized_storage->as ()); + if (isReplicationSlotExist(tx, start_lsn, /* temporary */true)) + dropReplicationSlot(tx, /* temporary */true); - auto table_id = materialized_storage->getNestedStorageID(); - auto temp_table_id = temp_nested_storage->getStorageID(); + createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true); - LOG_TRACE(log, "Starting background update of table {}.{} ({}) with table {}.{} ({})", - table_id.database_name, table_id.table_name, toString(table_id.uuid), - temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); - - auto ast_rename = std::make_shared(); - ASTRenameQuery::Element elem + for (const auto & [relation_id, table_name] : relation_data) { - ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, - ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} - }; - ast_rename->elements.push_back(std::move(elem)); - ast_rename->exchange = true; + auto storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context); + auto * materialized_storage = storage->as (); - auto nested_context = materialized_storage->getNestedTableContext(); + /// If for some reason this temporary table already exists - also drop it. + auto temp_materialized_storage = materialized_storage->createTemporary(); - try - { - auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); - InterpreterRenameQuery(ast_rename, nested_context).execute(); + /// This snapshot is valid up to the end of the transaction, which exported it. + StoragePtr temp_nested_storage = loadFromSnapshot(snapshot_name, table_name, + temp_materialized_storage->as ()); + auto table_id = materialized_storage->getNestedStorageID(); + auto temp_table_id = temp_nested_storage->getStorageID(); + + LOG_TRACE(log, "Starting background update of table {} with table {}", + table_id.getNameForLogs(), temp_table_id.getNameForLogs()); + + auto ast_rename = std::make_shared(); + ASTRenameQuery::Element elem { - auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), nested_context); - auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); - auto nested_table_id = nested_storage->getStorageID(); + ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, + ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} + }; + ast_rename->elements.push_back(std::move(elem)); + ast_rename->exchange = true; - materialized_storage->setNestedStorageID(nested_table_id); - nested_storage = materialized_storage->prepare(); + auto nested_context = materialized_storage->getNestedTableContext(); - auto nested_storage_metadata = nested_storage->getInMemoryMetadataPtr(); - auto nested_sample_block = nested_storage_metadata->getSampleBlock(); - LOG_TRACE(log, "Updated table {}.{} ({}). New structure: {}", - nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid), nested_sample_block.dumpStructure()); + try + { + auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + InterpreterRenameQuery(ast_rename, nested_context).execute(); - auto materialized_storage_metadata = nested_storage->getInMemoryMetadataPtr(); - auto materialized_sample_block = materialized_storage_metadata->getSampleBlock(); + { + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), + nested_context); + auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + auto nested_table_id = nested_storage->getStorageID(); - assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); + materialized_storage->setNestedStorageID(nested_table_id); + nested_storage = materialized_storage->prepare(); - /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. - consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); + auto nested_storage_metadata = nested_storage->getInMemoryMetadataPtr(); + auto nested_sample_block = nested_storage_metadata->getSampleBlock(); + LOG_TRACE(log, "Updated table {}. New structure: {}", + nested_table_id.getNameForLogs(), nested_sample_block.dumpStructure()); + + auto materialized_storage_metadata = nested_storage->getInMemoryMetadataPtr(); + auto materialized_sample_block = materialized_storage_metadata->getSampleBlock(); + + assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); + + /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. + consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); + } + + LOG_DEBUG(log, "Dropping table {}", temp_table_id.getNameForLogs()); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true); } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } - LOG_DEBUG(log, "Dropping table {}.{} ({})", temp_table_id.database_name, temp_table_id.table_name, toString(temp_table_id.uuid)); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + dropReplicationSlot(tx, /* temporary */true); + tx.commit(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } - - dropReplicationSlot(tx, /* temporary */true); - tx.commit(); } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 6c919389392..6ae9ec31626 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -1,6 +1,6 @@ #pragma once -#include "MaterializePostgreSQLConsumer.h" +#include "MaterializedPostgreSQLConsumer.h" #include #include @@ -13,7 +13,7 @@ namespace DB /// exist in CH, it can be loaded via snapshot while stream is stopped and then comparing wal positions with /// current lsn and table start lsn. -class StorageMaterializePostgreSQL; +class StorageMaterializedPostgreSQL; class PostgreSQLReplicationHandler { @@ -26,7 +26,7 @@ public: ContextPtr context_, const size_t max_block_size_, bool allow_automatic_update_, - bool is_materialize_postgresql_database_, + bool is_materialized_postgresql_database_, const String tables_list = ""); /// Activate task to be run from a separate thread: wait until connection is available and call startReplication(). @@ -39,7 +39,7 @@ public: void shutdownFinal(); /// Add storage pointer to let handler know which tables it needs to keep in sync. - void addStorage(const std::string & table_name, StorageMaterializePostgreSQL * storage); + void addStorage(const std::string & table_name, StorageMaterializedPostgreSQL * storage); /// Fetch list of tables which are going to be replicated. Used for database engine. NameSet fetchRequiredTables(pqxx::connection & connection_); @@ -48,7 +48,7 @@ public: void startSynchronization(bool throw_on_error); private: - using MaterializedStorages = std::unordered_map; + using MaterializedStorages = std::unordered_map; /// Methods to manage Publication. @@ -74,7 +74,7 @@ private: void consumerFunc(); - StoragePtr loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializePostgreSQL * materialized_storage); + StoragePtr loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializedPostgreSQL * materialized_storage); void reloadFromSnapshot(const std::vector> & relation_data); @@ -95,8 +95,8 @@ private: /// This setting allows to reloas table in the background. bool allow_automatic_update = false; - /// To distinguish whether current replication handler belongs to a MaterializePostgreSQL database engine or single storage. - bool is_materialize_postgresql_database; + /// To distinguish whether current replication handler belongs to a MaterializedPostgreSQL database engine or single storage. + bool is_materialized_postgresql_database; /// A coma-separated list of tables, which are going to be replicated for database engine. By default, a whole database is replicated. String tables_list; @@ -107,7 +107,7 @@ private: std::shared_ptr connection; /// Replication consumer. Manages decoding of replication stream and syncing into tables. - std::shared_ptr consumer; + std::shared_ptr consumer; BackgroundSchedulePool::TaskHolder startup_task, consumer_task; @@ -118,7 +118,7 @@ private: /// 2. at replication startup bool new_publication_created = false; - /// MaterializePostgreSQL tables. Used for managing all operations with its internal nested tables. + /// MaterializedPostgreSQL tables. Used for managing all operations with its internal nested tables. MaterializedStorages materialized_storages; UInt64 milliseconds_to_wait; diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp similarity index 78% rename from src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp rename to src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 07d13ace7c2..52fe7be35a7 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -1,4 +1,4 @@ -#include "StorageMaterializePostgreSQL.h" +#include "StorageMaterializedPostgreSQL.h" #if USE_LIBPQXX #include @@ -40,7 +40,7 @@ static const auto TMP_SUFFIX = "_tmp"; /// For the case of single storage. -StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( +StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( const StorageID & table_id_, bool is_attach_, const String & remote_database_name, @@ -48,10 +48,10 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, ContextPtr context_, - std::unique_ptr replication_settings) + std::unique_ptr replication_settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , is_materialize_postgresql_database(false) + , is_materialized_postgresql_database(false) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) , nested_table_id(StorageID(table_id_.database_name, getNestedTableName())) @@ -59,7 +59,7 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( , is_attach(is_attach_) { if (table_id_.uuid == UUIDHelpers::Nil) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializePostgreSQL is allowed only for Atomic database"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL is allowed only for Atomic database"); setInMemoryMetadata(storage_metadata); @@ -70,31 +70,31 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL( table_id_.database_name, connection_info, getContext(), - replication_settings->materialize_postgresql_max_block_size.value, - /* allow_automatic_update */ false, /* is_materialize_postgresql_database */false); + replication_settings->materialized_postgresql_max_block_size.value, + /* allow_automatic_update */ false, /* is_materialized_postgresql_database */false); } /// For the case of MaterializePosgreSQL database engine. /// It is used when nested ReplacingMergeeTree table has not yet be created by replication thread. /// In this case this storage can't be used for read queries. -StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(const StorageID & table_id_, ContextPtr context_) +StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , is_materialize_postgresql_database(true) + , is_materialized_postgresql_database(true) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) { } -/// Constructor for MaterializePostgreSQL table engine - for the case of MaterializePosgreSQL database engine. +/// Constructor for MaterializedPostgreSQL table engine - for the case of MaterializePosgreSQL database engine. /// It is used when nested ReplacingMergeeTree table has already been created by replication thread. /// This storage is ready to handle read queries. -StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(StoragePtr nested_storage_, ContextPtr context_) +StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(StoragePtr nested_storage_, ContextPtr context_) : IStorage(nested_storage_->getStorageID()) , WithContext(context_->getGlobalContext()) - , is_materialize_postgresql_database(true) + , is_materialized_postgresql_database(true) , has_nested(true) , nested_context(makeNestedTableContext(context_->getGlobalContext())) , nested_table_id(nested_storage_->getStorageID()) @@ -105,72 +105,82 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(StoragePtr nested_sto /// A temporary clone table might be created for current table in order to update its schema and reload /// all data in the background while current table will still handle read requests. -StoragePtr StorageMaterializePostgreSQL::createTemporary() const +StoragePtr StorageMaterializedPostgreSQL::createTemporary() const { auto table_id = getStorageID(); - auto new_context = Context::createCopy(context); + auto tmp_table_id = StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX); - return StorageMaterializePostgreSQL::create(StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX), new_context); + /// If for some reason it already exists - drop it. + auto tmp_storage = DatabaseCatalog::instance().tryGetTable(tmp_table_id, nested_context); + if (tmp_storage) + { + LOG_TRACE(&Poco::Logger::get("MaterializedPostgreSQLStorage"), "Temporary table {} already exists, dropping", tmp_table_id.getNameForLogs()); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), getContext(), tmp_table_id, /* no delay */true); + } + + auto new_context = Context::createCopy(context); + return StorageMaterializedPostgreSQL::create(tmp_table_id, new_context); } -StoragePtr StorageMaterializePostgreSQL::getNested() const +StoragePtr StorageMaterializedPostgreSQL::getNested() const { return DatabaseCatalog::instance().getTable(getNestedStorageID(), nested_context); } -StoragePtr StorageMaterializePostgreSQL::tryGetNested() const +StoragePtr StorageMaterializedPostgreSQL::tryGetNested() const { return DatabaseCatalog::instance().tryGetTable(getNestedStorageID(), nested_context); } -String StorageMaterializePostgreSQL::getNestedTableName() const +String StorageMaterializedPostgreSQL::getNestedTableName() const { auto table_id = getStorageID(); - if (is_materialize_postgresql_database) + if (is_materialized_postgresql_database) return table_id.table_name; return toString(table_id.uuid) + NESTED_TABLE_SUFFIX; } -StorageID StorageMaterializePostgreSQL::getNestedStorageID() const +StorageID StorageMaterializedPostgreSQL::getNestedStorageID() const { if (nested_table_id.has_value()) return nested_table_id.value(); auto table_id = getStorageID(); throw Exception(ErrorCodes::LOGICAL_ERROR, - "No storageID found for inner table. ({}.{}, {})", table_id.database_name, table_id.table_name, toString(table_id.uuid)); + "No storageID found for inner table. ({})", table_id.getNameForLogs()); } -void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) +void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) { const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); + auto table_id = getStorageID(); + auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName()); try { InterpreterCreateQuery interpreter(ast_create, nested_context); interpreter.execute(); - auto table_id = getStorageID(); - auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, getNestedTableName()), nested_context); - + auto nested_storage = DatabaseCatalog::instance().getTable(tmp_nested_table_id, nested_context); /// Save storage_id with correct uuid. nested_table_id = nested_storage->getStorageID(); } - catch (...) + catch (Exception & e) { + e.addMessage("while creating nested table: {}", tmp_nested_table_id.getNameForLogs()); tryLogCurrentException(__PRETTY_FUNCTION__); } } -std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) +std::shared_ptr StorageMaterializedPostgreSQL::makeNestedTableContext(ContextPtr from_context) { auto new_context = Context::createCopy(from_context); new_context->setInternalQuery(true); @@ -178,7 +188,7 @@ std::shared_ptr StorageMaterializePostgreSQL::makeNestedTableContext(Co } -StoragePtr StorageMaterializePostgreSQL::prepare() +StoragePtr StorageMaterializedPostgreSQL::prepare() { auto nested_table = getNested(); setInMemoryMetadata(nested_table->getInMemoryMetadata()); @@ -187,9 +197,10 @@ StoragePtr StorageMaterializePostgreSQL::prepare() } -void StorageMaterializePostgreSQL::startup() +void StorageMaterializedPostgreSQL::startup() { - if (!is_materialize_postgresql_database) + /// replication_handler != nullptr only in case of single table engine MaterializedPostgreSQL. + if (replication_handler) { replication_handler->addStorage(remote_table_name, this); @@ -202,7 +213,7 @@ void StorageMaterializePostgreSQL::startup() else { /// Start synchronization preliminary setup immediately and throw in case of failure. - /// It should be guaranteed that if MaterializePostgreSQL table was created successfully, then + /// It should be guaranteed that if MaterializedPostgreSQL table was created successfully, then /// its nested table was also created. replication_handler->startSynchronization(/* throw_on_error */ true); } @@ -210,25 +221,29 @@ void StorageMaterializePostgreSQL::startup() } -void StorageMaterializePostgreSQL::shutdown() +void StorageMaterializedPostgreSQL::shutdown() { if (replication_handler) replication_handler->shutdown(); } -void StorageMaterializePostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) +void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) { - if (replication_handler) - replication_handler->shutdownFinal(); + /// If it is a table with database engine MaterializedPostgreSQL - return, becuase delition of + /// internal tables is managed there. + if (is_materialized_postgresql_database) + return; + + replication_handler->shutdownFinal(); auto nested_table = getNested(); - if (nested_table && !is_materialize_postgresql_database) + if (nested_table) InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), no_delay); } -NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const +NamesAndTypesList StorageMaterializedPostgreSQL::getVirtuals() const { return NamesAndTypesList{ {"_sign", std::make_shared()}, @@ -237,7 +252,7 @@ NamesAndTypesList StorageMaterializePostgreSQL::getVirtuals() const } -Pipe StorageMaterializePostgreSQL::read( +Pipe StorageMaterializedPostgreSQL::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, @@ -253,7 +268,7 @@ Pipe StorageMaterializePostgreSQL::read( } -std::shared_ptr StorageMaterializePostgreSQL::getMaterializedColumnsDeclaration( +std::shared_ptr StorageMaterializedPostgreSQL::getMaterializedColumnsDeclaration( const String name, const String type, UInt64 default_value) { auto column_declaration = std::make_shared(); @@ -271,7 +286,7 @@ std::shared_ptr StorageMaterializePostgreSQL::getMateriali } -ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) const +ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) const { WhichDataType which(data_type); @@ -312,17 +327,17 @@ ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & da } -/// For single storage MaterializePostgreSQL get columns and primary key columns from storage definition. -/// For database engine MaterializePostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same +/// For single storage MaterializedPostgreSQL get columns and primary key columns from storage definition. +/// For database engine MaterializedPostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same /// transaction with snapshot, which is used for initial tables dump. -ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure) +ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure) { auto create_table_query = std::make_shared(); auto table_id = getStorageID(); create_table_query->table = getNestedTableName(); create_table_query->database = table_id.database_name; - if (is_materialize_postgresql_database) + if (is_materialized_postgresql_database) create_table_query->uuid = table_id.uuid; auto columns_declare_list = std::make_shared(); @@ -333,7 +348,7 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt const auto & columns = metadata_snapshot->getColumns(); NamesAndTypesList ordinary_columns_and_types; - if (!is_materialize_postgresql_database) + if (!is_materialized_postgresql_database) { ordinary_columns_and_types = columns.getOrdinary(); } @@ -416,19 +431,19 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableSt } -void registerStorageMaterializePostgreSQL(StorageFactory & factory) +void registerStorageMaterializedPostgreSQL(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; bool has_settings = args.storage_def->settings; - auto postgresql_replication_settings = std::make_unique(); + auto postgresql_replication_settings = std::make_unique(); if (has_settings) postgresql_replication_settings->loadFromQuery(*args.storage_def); if (engine_args.size() != 5) - throw Exception("Storage MaterializePostgreSQL requires 5 parameters: " + throw Exception("Storage MaterializedPostgreSQL requires 5 parameters: " "PostgreSQL('host:port', 'database', 'table', 'username', 'password'", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -443,7 +458,7 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); if (!args.storage_def->order_by) - throw Exception("Storage MaterializePostgreSQL needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Storage MaterializedPostgreSQL needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS); if (args.storage_def->primary_key) metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); @@ -462,14 +477,14 @@ void registerStorageMaterializePostgreSQL(StorageFactory & factory) engine_args[3]->as().value.safeGet(), engine_args[4]->as().value.safeGet()); - return StorageMaterializePostgreSQL::create( + return StorageMaterializedPostgreSQL::create( args.table_id, args.attach, remote_database, remote_table, connection_info, metadata, args.getContext(), std::move(postgresql_replication_settings)); }; factory.registerStorage( - "MaterializePostgreSQL", + "MaterializedPostgreSQL", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, diff --git a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h similarity index 73% rename from src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h rename to src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 622f40b14ad..5d18a0b16b7 100644 --- a/src/Storages/PostgreSQL/StorageMaterializePostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -6,7 +6,7 @@ #if USE_LIBPQXX #include "PostgreSQLReplicationHandler.h" -#include "MaterializePostgreSQLSettings.h" +#include "MaterializedPostgreSQLSettings.h" #include #include @@ -24,9 +24,9 @@ namespace DB { -/** Case of single MaterializePostgreSQL table engine. +/** Case of single MaterializedPostgreSQL table engine. * - * A user creates a table with engine MaterializePostgreSQL. Order by expression must be specified (needed for + * A user creates a table with engine MaterializedPostgreSQL. Order by expression must be specified (needed for * nested ReplacingMergeTree table). This storage owns its own replication handler, which loads table data * from PostgreSQL into nested ReplacingMergeTree table. If table is not created, but attached, replication handler * will not start loading-from-snapshot procedure, instead it will continue from last committed lsn. @@ -37,47 +37,47 @@ namespace DB **/ -/** Case of MaterializePostgreSQL database engine. +/** Case of MaterializedPostgreSQL database engine. * - * MaterializePostgreSQL table exists only in memory and acts as a wrapper for nested table, i.e. only provides an + * MaterializedPostgreSQL table exists only in memory and acts as a wrapper for nested table, i.e. only provides an * interface to work with nested table. Both tables share the same StorageID. * * Main table is never created or dropped via database method. The only way database engine interacts with - * MaterializePostgreSQL table - in tryGetTable() method, a MaterializePostgreSQL table is returned in order to wrap + * MaterializedPostgreSQL table - in tryGetTable() method, a MaterializedPostgreSQL table is returned in order to wrap * and redirect read requests. Set of such wrapper-tables is cached inside database engine. All other methods in * regard to materializePostgreSQL table are handled by replication handler. * * All database methods, apart from tryGetTable(), are devoted only to nested table. - * NOTE: It makes sense to allow rename method for MaterializePostgreSQL table via database method. + * NOTE: It makes sense to allow rename method for MaterializedPostgreSQL table via database method. * TODO: Make sure replication-to-table data channel is done only by relation_id. * * Also main table has the same InMemoryMetadata as its nested table, so if metadata of nested table changes - main table also has - * to update its metadata, because all read requests are passed to MaterializePostgreSQL table and then it redirects read + * to update its metadata, because all read requests are passed to MaterializedPostgreSQL table and then it redirects read * into nested table. * - * When there is a need to update table structure, there will be created a new MaterializePostgreSQL table with its own nested table, + * When there is a need to update table structure, there will be created a new MaterializedPostgreSQL table with its own nested table, * it will have updated table schema and all data will be loaded from scratch in the background, while previous table with outadted table * structure will still serve read requests. When data is loaded, nested tables will be swapped, metadata of metarialzied table will be * updated according to nested table. * **/ -class StorageMaterializePostgreSQL final : public shared_ptr_helper, public IStorage, WithContext +class StorageMaterializedPostgreSQL final : public shared_ptr_helper, public IStorage, WithContext { - friend struct shared_ptr_helper; + friend struct shared_ptr_helper; public: - StorageMaterializePostgreSQL(const StorageID & table_id_, ContextPtr context_); + StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_); - StorageMaterializePostgreSQL(StoragePtr nested_storage_, ContextPtr context_); + StorageMaterializedPostgreSQL(StoragePtr nested_storage_, ContextPtr context_); - String getName() const override { return "MaterializePostgreSQL"; } + String getName() const override { return "MaterializedPostgreSQL"; } void startup() override; void shutdown() override; - /// Used only for single MaterializePostgreSQL storage. + /// Used only for single MaterializedPostgreSQL storage. void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override; NamesAndTypesList getVirtuals() const override; @@ -102,7 +102,7 @@ public: StoragePtr tryGetNested() const; - /// Create a temporary MaterializePostgreSQL table with current_table_name + TMP_SUFFIX. + /// Create a temporary MaterializedPostgreSQL table with current_table_name + TMP_SUFFIX. /// An empty wrapper is returned - it does not have inMemory metadata, just acts as an empty wrapper over /// temporary nested, which will be created shortly after. StoragePtr createTemporary() const; @@ -120,7 +120,7 @@ public: StoragePtr prepare(); protected: - StorageMaterializePostgreSQL( + StorageMaterializedPostgreSQL( const StorageID & table_id_, bool is_attach_, const String & remote_database_name, @@ -128,7 +128,7 @@ protected: const postgres::ConnectionInfo & connection_info, const StorageInMemoryMetadata & storage_metadata, ContextPtr context_, - std::unique_ptr replication_settings); + std::unique_ptr replication_settings); private: static std::shared_ptr getMaterializedColumnsDeclaration( @@ -140,37 +140,36 @@ private: String getNestedTableName() const; - /// Not nullptr only for single MaterializePostgreSQL storage, because for MaterializePostgreSQL + /// Not nullptr only for single MaterializedPostgreSQL storage, because for MaterializedPostgreSQL /// database engine there is one replication handler for all tables. std::unique_ptr replication_handler; - /// Distinguish between single MaterilizePostgreSQL table engine and MaterializePostgreSQL database engine, + /// Distinguish between single MaterilizePostgreSQL table engine and MaterializedPostgreSQL database engine, /// because table with engine MaterilizePostgreSQL acts differently in each case. - bool is_materialize_postgresql_database = false; + bool is_materialized_postgresql_database = false; /// Will be set to `true` only once - when nested table was loaded by replication thread. - /// After that, it will never be changed. Needed for MaterializePostgreSQL database engine + /// After that, it will never be changed. Needed for MaterializedPostgreSQL database engine /// because there is an invariant - table exists only if its nested table exists, but nested /// table is not loaded immediately. It is made atomic, because it is accessed only by database engine, /// and updated by replication handler (only once). std::atomic has_nested = false; - /// Nested table context is a copy of global context, but contains query context with defined - /// ReplacingMergeTree storage in factoriesLog. This is needed to let database engine know - /// whether to access nested table or a wrapper over nested (materialized table). + /// Nested table context is a copy of global context, but modified to answer isInternalQuery() == true. + /// This is needed to let database engine know whether to access nested table or a wrapper over nested (materialized table). ContextMutablePtr nested_context; /// Save nested storageID to be able to fetch it. It is set once nested is created and will be /// updated only when nested is reloaded or renamed. std::optional nested_table_id; - /// Needed only for the case of single MaterializePostgreSQL storage - in order to make + /// Needed only for the case of single MaterializedPostgreSQL storage - in order to make /// delayed storage forwarding into replication handler. String remote_table_name; - /// Needed only for the case of single MaterializePostgreSQL storage, because in case of create + /// Needed only for the case of single MaterializedPostgreSQL storage, because in case of create /// query (not attach) initial setup will be done immediately and error message is thrown at once. - /// It results in the fact: single MaterializePostgreSQL storage is created only if its nested table is created. + /// It results in the fact: single MaterializedPostgreSQL storage is created only if its nested table is created. /// In case of attach - this setup will be done in a separate thread in the background. It will also /// be checked for nested table and attempted to load it if it does not exist for some reason. bool is_attach = true; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index b1c8f17e2b8..0b302ee437a 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -60,7 +60,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory); #if USE_LIBPQXX void registerStoragePostgreSQL(StorageFactory & factory); -void registerStorageMaterializePostgreSQL(StorageFactory & factory); +void registerStorageMaterializedPostgreSQL(StorageFactory & factory); #endif #if USE_MYSQL || USE_LIBPQXX @@ -122,7 +122,7 @@ void registerStorages() #if USE_LIBPQXX registerStoragePostgreSQL(factory); - registerStorageMaterializePostgreSQL(factory); + registerStorageMaterializedPostgreSQL(factory); #endif #if USE_MYSQL || USE_LIBPQXX diff --git a/tests/integration/test_postgresql_replica_database_engine/configs/users.xml b/tests/integration/test_postgresql_replica_database_engine/configs/users.xml index 74d2737c821..1cdece49459 100644 --- a/tests/integration/test_postgresql_replica_database_engine/configs/users.xml +++ b/tests/integration/test_postgresql_replica_database_engine/configs/users.xml @@ -2,7 +2,7 @@ - 1 + 1 diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index d1e590704fd..f0ed06c7ad3 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -62,7 +62,7 @@ def create_materialized_db(ip, port, materialized_database='test_database', postgres_database='postgres_database', settings=[]): - create_query = "CREATE DATABASE {} ENGINE = MaterializePostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')".format(materialized_database, ip, port, postgres_database) + create_query = "CREATE DATABASE {} ENGINE = MaterializedPostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')".format(materialized_database, ip, port, postgres_database) if len(settings) > 0: create_query += " SETTINGS " for i in range(len(settings)): @@ -115,16 +115,16 @@ def assert_nested_table_is_created(table_name, materialized_database='test_datab @pytest.mark.timeout(30) def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'): - assert_nested_table_is_created(table_name, materialized_database) + assert_nested_table_is_created(table_name, materialized_database) - expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) + expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) + result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) + + while result != expected: + time.sleep(0.5) result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) - while result != expected: - time.sleep(0.5) - result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) - - assert(result == expected) + assert(result == expected) @pytest.fixture(scope="module") @@ -328,7 +328,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, - settings=["materialize_postgresql_tables_list = '{}'".format(publication_tables)]) + settings=["materialized_postgresql_tables_list = '{}'".format(publication_tables)]) assert 'test_database' in instance.query('SHOW DATABASES') time.sleep(1) @@ -391,7 +391,7 @@ def test_clickhouse_restart(started_cluster): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i)) - instance.query("CREATE DATABASE test_database ENGINE = MaterializePostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") + instance.query("CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')") for i in range(NUM_TABLES): table_name = 'postgresql_replica_{}'.format(i) @@ -449,7 +449,7 @@ def test_table_schema_changes(started_cluster): create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, - settings=["materialize_postgresql_allow_automatic_update = 1"]) + settings=["materialized_postgresql_allow_automatic_update = 1"]) 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)) @@ -608,7 +608,7 @@ def test_virtual_columns(started_cluster): create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, - settings=["materialize_postgresql_allow_automatic_update = 1"]) + settings=["materialized_postgresql_allow_automatic_update = 1"]) assert_nested_table_is_created('postgresql_replica_0') instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number from numbers(10)") check_tables_are_synchronized('postgresql_replica_0'); diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index e448cfc8e99..d5bb4a85733 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -40,7 +40,7 @@ def create_clickhouse_postgres_db(ip, port, name='postgres_database'): def create_materialized_table(ip, port): instance.query(''' CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) - ENGINE = MaterializePostgreSQL( + ENGINE = MaterializedPostgreSQL( '{}:{}', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') PRIMARY KEY key; '''.format(ip, port)) From b6b79244136b8f3a9a3372bd67d2168616c67a60 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Jun 2021 10:20:33 +0300 Subject: [PATCH 097/105] Update DatabaseAtomic.cpp --- src/Databases/DatabaseAtomic.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index d7d423cfea6..48777d92a05 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -112,7 +112,8 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na auto * storage = tryGetTable(table_name, local_context).get(); /// Remove the inner table (if any) to avoid deadlock /// (due to attempt to execute DROP from the worker thread) - storage->dropInnerTableIfAny(no_delay, local_context); + if (storage) + storage->dropInnerTableIfAny(no_delay, local_context); String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop; From a96c7a3a70bad52281a6253f58253ba945670620 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Jun 2021 17:32:03 +0300 Subject: [PATCH 098/105] Update DatabaseAtomic.cpp --- src/Databases/DatabaseAtomic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 48777d92a05..9ab041ee36f 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -109,7 +109,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) { - auto * storage = tryGetTable(table_name, local_context).get(); + auto storage = tryGetTable(table_name, local_context).get(); /// Remove the inner table (if any) to avoid deadlock /// (due to attempt to execute DROP from the worker thread) if (storage) From ff8c44179c98e48dc78fa9a1d0ff7d5d6dcc6728 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Jun 2021 17:37:22 +0300 Subject: [PATCH 099/105] Update DatabaseAtomic.cpp --- src/Databases/DatabaseAtomic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 9ab041ee36f..b69b74451c7 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -109,7 +109,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) { - auto storage = tryGetTable(table_name, local_context).get(); + auto storage = tryGetTable(table_name, local_context); /// Remove the inner table (if any) to avoid deadlock /// (due to attempt to execute DROP from the worker thread) if (storage) From da70f85d4e686d13ca93b3bb4a645126797c9549 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Jun 2021 23:11:46 +0000 Subject: [PATCH 100/105] More tests --- src/Core/PostgreSQL/Connection.cpp | 36 +++- src/Core/PostgreSQL/Connection.h | 14 ++ .../MaterializedPostgreSQLConsumer.cpp | 39 +++-- .../MaterializedPostgreSQLConsumer.h | 2 +- .../test.py | 164 ++++++++++++++++++ .../test_storage_postgresql_replica/test.py | 5 - 6 files changed, 235 insertions(+), 25 deletions(-) diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp index 71dfa7ea305..d69856a392e 100644 --- a/src/Core/PostgreSQL/Connection.cpp +++ b/src/Core/PostgreSQL/Connection.cpp @@ -6,6 +6,7 @@ namespace postgres Connection::Connection(const ConnectionInfo & connection_info_, bool replication_, size_t num_tries_) : connection_info(connection_info_), replication(replication_), num_tries(num_tries_) + , log(&Poco::Logger::get("PostgreSQLReplicaConnection")) { if (replication) { @@ -25,8 +26,7 @@ void Connection::execWithRetry(const std::function } catch (const pqxx::broken_connection & e) { - LOG_DEBUG(&Poco::Logger::get("PostgreSQLReplicaConnection"), - "Cannot execute query due to connection failure, attempt: {}/{}. (Message: {})", + LOG_DEBUG(log, "Cannot execute query due to connection failure, attempt: {}/{}. (Message: {})", try_no, num_tries, e.what()); if (try_no == num_tries) @@ -38,18 +38,36 @@ void Connection::execWithRetry(const std::function pqxx::connection & Connection::getRef() { connect(); + assert(connection != nulptr); return *connection; } +void Connection::tryUpdateConnection() +{ + try + { + updateConnection(); + } + catch (const pqxx::broken_connection & e) + { + LOG_ERROR(log, "Unable to update connection: {}", e.what()); + } +} + +void Connection::updateConnection() +{ + if (connection) + connection->close(); + /// Always throws if there is no connection. + connection = std::make_unique(connection_info.first); + if (replication) + connection->set_variable("default_transaction_isolation", "'repeatable read'"); + LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second); +} + void Connection::connect() { if (!connection || !connection->is_open()) - { - /// Always throws if there is no connection. - connection = std::make_unique(connection_info.first); - if (replication) - connection->set_variable("default_transaction_isolation", "'repeatable read'"); - LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second); - } + updateConnection(); } } diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h index 00cf0c737f6..e01de419c17 100644 --- a/src/Core/PostgreSQL/Connection.h +++ b/src/Core/PostgreSQL/Connection.h @@ -4,6 +4,14 @@ #include #include +/* Methods to work with PostgreSQL connection object. + * Should only be used in case there has to be a single connection object, which + * is long-lived and there are no concurrent connection queries. + * Now only use case - for replication handler for replication from PostgreSQL. + * In all other integration engine use pool with failover. + **/ + +namespace Poco { class Logger; } namespace postgres { @@ -21,13 +29,19 @@ public: void connect(); + void tryUpdateConnection(); + const ConnectionInfo & getConnectionInfo() { return connection_info; } private: + void updateConnection(); + ConnectionPtr connection; ConnectionInfo connection_info; bool replication; size_t num_tries; + + Poco::Logger * log; }; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 390ad996e4e..2d98fc79c40 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -455,7 +455,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl } -void MaterializedPostgreSQLConsumer::syncTables(std::shared_ptr tx) +void MaterializedPostgreSQLConsumer::syncTables() { try { @@ -488,6 +488,7 @@ void MaterializedPostgreSQLConsumer::syncTables(std::shared_ptr(connection->getRef()); current_lsn = advanceLSN(tx); tables_to_sync.clear(); tx->commit(); @@ -569,12 +570,11 @@ void MaterializedPostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const /// Read binary changes from replication slot via COPY command (starting from current lsn in a slot). bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() { - std::shared_ptr tx; bool slot_empty = true; try { - tx = std::make_shared(connection->getRef()); + auto tx = std::make_shared(connection->getRef()); /// Read up to max_block_size rows changes (upto_n_changes parameter). It might return larger number as the limit /// is checked only after each transaction block. @@ -611,6 +611,17 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } } + catch (const Exception &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + return false; + } + catch (const pqxx::broken_connection & e) + { + LOG_ERROR(log, "Connection error: {}", e.what()); + connection->tryUpdateConnection(); + return false; + } catch (const pqxx::sql_error & e) { /// For now sql replication interface is used and it has the problem that it registers relcache @@ -628,14 +639,24 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() LOG_ERROR(log, "Conversion error: {}", e.what()); return false; } - catch (const pqxx::broken_connection & e) + catch (const pqxx::statement_completion_unknown & e) { - LOG_ERROR(log, "Connection error: {}", e.what()); + LOG_ERROR(log, "Unknown statement completion: {}", e.what()); return false; } - catch (const Exception &) + catch (const pqxx::in_doubt_error & e) { - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_ERROR(log, "PostgreSQL library has some doubts: {}", e.what()); + return false; + } + catch (const pqxx::internal_error & e) + { + LOG_ERROR(log, "PostgreSQL library internal error: {}", e.what()); + return false; + } + catch (const pqxx::conversion_overrun & e) + { + LOG_ERROR(log, "PostgreSQL library conversion overflow: {}", e.what()); return false; } catch (...) @@ -653,9 +674,7 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() } if (!tables_to_sync.empty()) - { - syncTables(tx); - } + syncTables(); return true; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 59feb60a21a..8f3224784f1 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -39,7 +39,7 @@ private: /// Read approximarely up to max_block_size changes from WAL. bool readFromReplicationSlot(); - void syncTables(std::shared_ptr tx); + void syncTables(); String advanceLSN(std::shared_ptr ntx); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index f0ed06c7ad3..4be2ab01d6b 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -734,6 +734,170 @@ def test_concurrent_transactions(started_cluster): drop_materialized_db() +@pytest.mark.timeout(320) +def test_abrupt_connection_loss_while_heavy_replication(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) + cursor = conn.cursor() + NUM_TABLES = 6 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + + def transaction(thread_id): + if thread_id % 2: + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=True) + else: + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=False) + cursor_ = conn.cursor() + for query in queries: + cursor_.execute(query.format(thread_id)) + print('thread {}, query {}'.format(thread_id, query)) + if thread_id % 2 == 0: + conn.commit() + + threads = [] + threads_num = 6 + for i in range(threads_num): + threads.append(threading.Thread(target=transaction, args=(i,))) + + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) + + for thread in threads: + time.sleep(random.uniform(0, 0.5)) + thread.start() + + # Join here because it takes time for data to reach wal + for thread in threads: + thread.join() + time.sleep(1) + started_cluster.pause_container('postgres1') + + for i in range(NUM_TABLES): + result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) + print(result) # Just debug + + started_cluster.unpause_container('postgres1') + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) + print(result) # Just debug + + drop_materialized_db() + + +def test_drop_database_while_replication_startup_not_finished(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + 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): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table(cursor, table_name); + instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(100000)".format(table_name)) + + for i in range(6): + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port) + time.sleep(0.5 * i) + drop_materialized_db() + + +def test_restart_server_while_replication_startup_not_finished(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + 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): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table(cursor, table_name); + instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(100000)".format(table_name)) + + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port) + time.sleep(0.5) + instance.restart_clickhouse() + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + drop_materialized_db() + + +# Something not ok with this test, need to investigate. +@pytest.mark.timeout(320) +def test_abrupt_server_restart_while_heavy_replication(started_cluster): + return +# instance.query("DROP DATABASE IF EXISTS test_database") +# conn = get_postgres_conn(ip=started_cluster.postgres_ip, +# port=started_cluster.postgres_port, +# database=True) +# cursor = conn.cursor() +# NUM_TABLES = 6 +# +# for i in range(NUM_TABLES): +# create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); +# +# def transaction(thread_id): +# if thread_id % 2: +# conn = get_postgres_conn(ip=started_cluster.postgres_ip, +# port=started_cluster.postgres_port, +# database=True, auto_commit=True) +# else: +# conn = get_postgres_conn(ip=started_cluster.postgres_ip, +# port=started_cluster.postgres_port, +# database=True, auto_commit=False) +# cursor_ = conn.cursor() +# for query in queries: +# cursor_.execute(query.format(thread_id)) +# print('thread {}, query {}'.format(thread_id, query)) +# if thread_id % 2 == 0: +# conn.commit() +# +# threads = [] +# threads_num = 6 +# for i in range(threads_num): +# threads.append(threading.Thread(target=transaction, args=(i,))) +# +# create_materialized_db(ip=started_cluster.postgres_ip, +# port=started_cluster.postgres_port) +# +# for thread in threads: +# time.sleep(random.uniform(0, 0.5)) +# thread.start() +# +# # Join here because it takes time for data to reach wal +# for thread in threads: +# thread.join() +# time.sleep(1) +# instance.restart_clickhouse() +# +# for i in range(NUM_TABLES): +# result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) +# print(result) # Just debug +# +# for i in range(NUM_TABLES): +# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); +# +# for i in range(NUM_TABLES): +# result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) +# print(result) # Just debug +# +# drop_materialized_db() + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index d5bb4a85733..31e211d1938 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -77,11 +77,6 @@ def started_cluster(): finally: cluster.shutdown() -@pytest.fixture(autouse=True) -def postgresql_setup_teardown(): - yield # run test - instance.query('DROP TABLE IF EXISTS test.postgresql_replica') - @pytest.mark.timeout(320) def test_initial_load_from_snapshot(started_cluster): From f01c8edbff57717a027184c559d899148755066d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Jul 2021 07:33:58 +0000 Subject: [PATCH 101/105] Fix access to tables after restart --- .../DatabaseMaterializedPostgreSQL.cpp | 6 + .../DatabaseMaterializedPostgreSQL.h | 2 + src/Interpreters/DatabaseCatalog.cpp | 12 ++ .../MaterializedPostgreSQLConsumer.cpp | 47 +++++--- .../PostgreSQLReplicationHandler.cpp | 3 +- .../StorageMaterializedPostgreSQL.cpp | 3 + .../test.py | 109 ++++++++---------- 7 files changed, 103 insertions(+), 79 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 37a464c6cda..4cfb5a4d137 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -153,6 +153,12 @@ void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const } +void DatabaseMaterializedPostgreSQL::shutdown() +{ + stopReplication(); +} + + void DatabaseMaterializedPostgreSQL::stopReplication() { if (replication_handler) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 0a60f47cbe4..f998a0c54de 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -57,6 +57,8 @@ public: void stopReplication(); + void shutdown() override; + private: void startSynchronization(); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 4ed4f258b29..0d0c82f1abc 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -28,6 +28,10 @@ # include #endif +#if USE_LIBPQXX +# include +#endif + namespace fs = std::filesystem; namespace CurrentMetrics @@ -234,6 +238,13 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return {}; } +#if USE_LIBPQXX + if (!context_->isInternalQuery() && (db_and_table.first->getEngineName() == "MaterializedPostgreSQL")) + { + db_and_table.second = std::make_shared(std::move(db_and_table.second), getContext()); + } +#endif + #if USE_MYSQL /// It's definitely not the best place for this logic, but behaviour must be consistent with DatabaseMaterializeMySQL::tryGetTable(...) if (db_and_table.first->getEngineName() == "MaterializeMySQL") @@ -245,6 +256,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return db_and_table; } + if (table_id.database_name == TEMPORARY_DATABASE) { /// For temporary tables UUIDs are set in Context::resolveStorageID(...). diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 2d98fc79c40..f80353f5586 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -38,6 +38,12 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( , allow_automatic_update(allow_automatic_update_) , storages(storages_) { + final_lsn = start_lsn; + auto tx = std::make_shared(connection->getRef()); + current_lsn = advanceLSN(tx); + LOG_TRACE(log, "Starting replication. LSN: {} (last: {})", getLSNValue(current_lsn), getLSNValue(final_lsn)); + tx->commit(); + for (const auto & [table_name, storage] : storages) { buffers.emplace(table_name, Buffer(storage)); @@ -298,7 +304,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl /// In this case, first comes a tuple with old replica identity indexes and all other values will come as /// nulls. Then comes a full new row. case 'K': [[fallthrough]]; - /// Old row. Only if replica identity is set to full. Does notreally make sense to use it as + /// Old row. Only if replica identity is set to full. Does not really make sense to use it as /// it is much more efficient to use replica identity index, but support all possible cases. case 'O': { @@ -371,7 +377,9 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl if (storages.find(relation_name) == storages.end()) { markTableAsSkipped(relation_id, relation_name); - LOG_ERROR(log, "Storage for table {} does not exist, but is included in replication stream", relation_name); + LOG_ERROR(log, + "Storage for table {} does not exist, but is included in replication stream. (Storages number: {})", + relation_name, storages.size()); return; } @@ -468,13 +476,13 @@ void MaterializedPostgreSQLConsumer::syncTables() { auto storage = storages[table_name]; + auto insert_context = Context::createCopy(context); + insert_context->setInternalQuery(true); + auto insert = std::make_shared(); insert->table_id = storage->getStorageID(); insert->columns = buffer.columnsAST; - auto insert_context = Context::createCopy(context); - insert_context->setInternalQuery(true); - InterpreterInsertQuery interpreter(insert, insert_context, true); auto block_io = interpreter.execute(); OneBlockInputStream input(result_rows); @@ -505,10 +513,8 @@ String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptrexec(query_str)}; - if (!result.empty()) - return result[0][0].as(); - - LOG_TRACE(log, "Advanced LSN up to: {}", final_lsn); + final_lsn = result[0][0].as(); + LOG_TRACE(log, "Advanced LSN up to: {}", getLSNValue(final_lsn)); return final_lsn; } @@ -552,18 +558,21 @@ void MaterializedPostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const /// Empty lsn string means - continue waiting for valid lsn. skip_list.insert({relation_id, ""}); - /// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream - /// and it receives first data after update. - schema_data.erase(relation_id); + if (storages.count(relation_name)) + { + /// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream + /// and it receives first data after update. + schema_data.erase(relation_id); - /// Clear table buffer. - auto & buffer = buffers.find(relation_name)->second; - buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); + /// Clear table buffer. + auto & buffer = buffers.find(relation_name)->second; + buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); - if (allow_automatic_update) - LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id); - else - LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name, relation_id); + if (allow_automatic_update) + LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id); + else + LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name, relation_id); + } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 74e3a2fb965..1d8ab04cfec 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -156,7 +156,6 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// and pass them to replication consumer. else { - LOG_TRACE(log, "Loading {} tables...", materialized_storages.size()); for (const auto & [table_name, storage] : materialized_storages) { auto * materialized_storage = storage->as (); @@ -174,12 +173,14 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) throw; } } + LOG_TRACE(log, "Loaded {} tables", nested_storages.size()); } tx.commit(); /// Pass current connection to consumer. It is not std::moved implicitly, but a shared_ptr is passed. /// Consumer and replication handler are always executed one after another (not concurrently) and share the same connection. + /// (Apart from the case, when shutdownFinal is called). /// Handler uses it only for loadFromSnapshot and shutdown methods. consumer = std::make_shared( context, diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 52fe7be35a7..903bab2b12c 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -225,6 +225,9 @@ void StorageMaterializedPostgreSQL::shutdown() { if (replication_handler) replication_handler->shutdown(); + auto nested = getNested(); + if (nested) + nested->shutdown(); } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 4be2ab01d6b..dca9be87311 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -101,7 +101,8 @@ queries = [ 'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;', "UPDATE postgresql_replica_{} SET key=key+10000000", 'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;', - 'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;'] + 'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;' + ] @pytest.mark.timeout(30) @@ -836,66 +837,56 @@ def test_restart_server_while_replication_startup_not_finished(started_cluster): drop_materialized_db() -# Something not ok with this test, need to investigate. @pytest.mark.timeout(320) def test_abrupt_server_restart_while_heavy_replication(started_cluster): - return -# instance.query("DROP DATABASE IF EXISTS test_database") -# conn = get_postgres_conn(ip=started_cluster.postgres_ip, -# port=started_cluster.postgres_port, -# database=True) -# cursor = conn.cursor() -# NUM_TABLES = 6 -# -# for i in range(NUM_TABLES): -# create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); -# -# def transaction(thread_id): -# if thread_id % 2: -# conn = get_postgres_conn(ip=started_cluster.postgres_ip, -# port=started_cluster.postgres_port, -# database=True, auto_commit=True) -# else: -# conn = get_postgres_conn(ip=started_cluster.postgres_ip, -# port=started_cluster.postgres_port, -# database=True, auto_commit=False) -# cursor_ = conn.cursor() -# for query in queries: -# cursor_.execute(query.format(thread_id)) -# print('thread {}, query {}'.format(thread_id, query)) -# if thread_id % 2 == 0: -# conn.commit() -# -# threads = [] -# threads_num = 6 -# for i in range(threads_num): -# threads.append(threading.Thread(target=transaction, args=(i,))) -# -# create_materialized_db(ip=started_cluster.postgres_ip, -# port=started_cluster.postgres_port) -# -# for thread in threads: -# time.sleep(random.uniform(0, 0.5)) -# thread.start() -# -# # Join here because it takes time for data to reach wal -# for thread in threads: -# thread.join() -# time.sleep(1) -# instance.restart_clickhouse() -# -# for i in range(NUM_TABLES): -# result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) -# print(result) # Just debug -# -# for i in range(NUM_TABLES): -# check_tables_are_synchronized('postgresql_replica_{}'.format(i)); -# -# for i in range(NUM_TABLES): -# result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) -# print(result) # Just debug -# -# drop_materialized_db() + instance.query("DROP DATABASE IF EXISTS test_database") + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) + cursor = conn.cursor() + NUM_TABLES = 2 + + for i in range(NUM_TABLES): + create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); + + def transaction(thread_id): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=True) + cursor_ = conn.cursor() + for query in queries: + cursor_.execute(query.format(thread_id)) + print('thread {}, query {}'.format(thread_id, query)) + + threads = [] + threads_num = 2 + for i in range(threads_num): + threads.append(threading.Thread(target=transaction, args=(i,))) + + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) + + for thread in threads: + time.sleep(random.uniform(0, 0.5)) + thread.start() + + # Join here because it takes time for data to reach wal + for thread in threads: + thread.join() + instance.restart_clickhouse() + + for i in range(NUM_TABLES): + result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) + print(result) # Just debug + + for i in range(NUM_TABLES): + check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + + for i in range(NUM_TABLES): + result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i)) + print(result) # Just debug + + drop_materialized_db() if __name__ == '__main__': From 4ad0f45f0e3c1d5ac103d255883a66b1d516c0fd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Jul 2021 08:20:13 +0000 Subject: [PATCH 102/105] Some more tests --- .../materialized-postgresql.md | 4 +- .../integrations/materialized-postgresql.md | 4 +- .../StorageMaterializedPostgreSQL.cpp | 2 +- src/Storages/StorageMaterializeMySQL.cpp | 60 +--------- .../test.py | 17 ++- .../test_storage_postgresql_replica/test.py | 109 +++++++++++++++++- 6 files changed, 129 insertions(+), 67 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index dde3886f694..6b8c6e86952 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -66,6 +66,6 @@ WHERE oid = 'postgres_table'::regclass; ``` -## WARNINGS {#warnings} +## Warning {#warning} -1. **TOAST** values convertions is not supported. Default value for the data type will be used. +1. **TOAST** values convertion is not supported. Default value for the data type will be used. diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 8c645d03e13..30cc6a11c48 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -41,6 +41,6 @@ SELECT key, value, _version FROM test.postgresql_replica; ``` -## WARNINGS {#warnings} +## Warning {#warning} -1. **TOAST** values convertions is not supported. Default value for the data type will be used. +1. **TOAST** values convertion is not supported. Default value for the data type will be used. diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 903bab2b12c..252059f606d 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -233,7 +233,7 @@ void StorageMaterializedPostgreSQL::shutdown() void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context) { - /// If it is a table with database engine MaterializedPostgreSQL - return, becuase delition of + /// If it is a table with database engine MaterializedPostgreSQL - return, because delition of /// internal tables is managed there. if (is_materialized_postgresql_database) return; diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 8e6f2e1ad63..5b371fe3fb8 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -22,6 +22,7 @@ #include #include +#include #include namespace DB @@ -37,7 +38,7 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora Pipe StorageMaterializeMySQL::read( const Names & column_names, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum processed_stage, @@ -46,61 +47,8 @@ Pipe StorageMaterializeMySQL::read( { /// If the background synchronization thread has exception. rethrowSyncExceptionIfNeed(database); - - NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); - auto lock = nested_storage->lockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); - const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); - - Block nested_header = nested_metadata->getSampleBlock(); - ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); - ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); - - if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) - { - auto & tables_in_select_query = select_query->tables()->as(); - - if (!tables_in_select_query.children.empty()) - { - auto & tables_element = tables_in_select_query.children[0]->as(); - - if (tables_element.table_expression) - tables_element.table_expression->as().final = true; - } - } - - String filter_column_name; - Names require_columns_name = column_names; - ASTPtr expressions = std::make_shared(); - if (column_names_set.empty() || !column_names_set.count(sign_column.name)) - { - require_columns_name.emplace_back(sign_column.name); - - const auto & sign_column_name = std::make_shared(sign_column.name); - const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); - - expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); - filter_column_name = expressions->children.back()->getColumnName(); - - for (const auto & column_name : column_names) - expressions->children.emplace_back(std::make_shared(column_name)); - } - - Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); - pipe.addTableLock(lock); - - if (!expressions->children.empty() && !pipe.empty()) - { - Block pipe_header = pipe.getHeader(); - auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); - ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true /* add_aliases */, false /* project_result */); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, expression_actions, filter_column_name, false); - }); - } - - return pipe; + return readFinalFromNestedStorage(nested_storage, column_names, metadata_snapshot, + query_info, context, processed_stage, max_block_size, num_streams); } NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index dca9be87311..d13a7501b35 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -844,22 +844,29 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): port=started_cluster.postgres_port, database=True) cursor = conn.cursor() - NUM_TABLES = 2 + NUM_TABLES = 6 for i in range(NUM_TABLES): create_postgres_table(cursor, 'postgresql_replica_{}'.format(i)); def transaction(thread_id): - conn = get_postgres_conn(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, auto_commit=True) + if thread_id % 2: + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=True) + else: + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True, auto_commit=False) cursor_ = conn.cursor() for query in queries: cursor_.execute(query.format(thread_id)) print('thread {}, query {}'.format(thread_id, query)) + if thread_id % 2 == 0: + conn.commit() threads = [] - threads_num = 2 + threads_num = 6 for i in range(threads_num): threads.append(threading.Thread(target=transaction, args=(i,))) diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 31e211d1938..4602d567b46 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -8,6 +8,8 @@ from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.test_tools import TSV +import threading + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True, stay_alive=True) @@ -16,6 +18,40 @@ postgres_table_template = """ key Integer NOT NULL, value Integer, PRIMARY KEY(key)) """ +queries = [ + 'INSERT INTO postgresql_replica select i, i from generate_series(0, 10000) as t(i);', + 'DELETE FROM postgresql_replica WHERE (value*value) % 3 = 0;', + 'UPDATE postgresql_replica SET value = value + 125 WHERE key % 2 = 0;', + "UPDATE postgresql_replica SET key=key+20000 WHERE key%2=0", + 'INSERT INTO postgresql_replica select i, i from generate_series(40000, 50000) as t(i);', + 'DELETE FROM postgresql_replica WHERE key % 10 = 0;', + 'UPDATE postgresql_replica SET value = value + 101 WHERE key % 2 = 1;', + "UPDATE postgresql_replica SET key=key+80000 WHERE key%2=1", + 'DELETE FROM postgresql_replica WHERE value % 2 = 0;', + 'UPDATE postgresql_replica SET value = value + 2000 WHERE key % 5 = 0;', + 'INSERT INTO postgresql_replica select i, i from generate_series(200000, 250000) as t(i);', + 'DELETE FROM postgresql_replica WHERE value % 3 = 0;', + 'UPDATE postgresql_replica SET value = value * 2 WHERE key % 3 = 0;', + "UPDATE postgresql_replica SET key=key+500000 WHERE key%2=1", + 'INSERT INTO postgresql_replica select i, i from generate_series(1000000, 1050000) as t(i);', + 'DELETE FROM postgresql_replica WHERE value % 9 = 2;', + "UPDATE postgresql_replica SET key=key+10000000", + 'UPDATE postgresql_replica SET value = value + 2 WHERE key % 3 = 1;', + 'DELETE FROM postgresql_replica WHERE value%5 = 0;' + ] + + +@pytest.mark.timeout(30) +def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database'): + expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) + result = instance.query('select * from test.{} order by {};'.format(table_name, order_by)) + + while result != expected: + time.sleep(0.5) + result = instance.query('select * from test.{} order by {};'.format(table_name, order_by)) + + assert(result == expected) + def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database'): if database == True: conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format(ip, port, database_name) @@ -28,7 +64,6 @@ def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name= conn.autocommit = True return conn - def create_postgres_db(cursor, name): cursor.execute("CREATE DATABASE {}".format(name)) @@ -487,6 +522,78 @@ def test_virtual_columns(started_cluster): cursor.execute('DROP TABLE postgresql_replica;') +def test_abrupt_connection_loss_while_heavy_replication(started_cluster): + instance.query("DROP DATABASE IF EXISTS test_database") + 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'); + + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) + + for i in range(len(queries)): + query = queries[i] + cursor.execute(query) + print('query {}'.format(query)) + + started_cluster.pause_container('postgres1') + + result = instance.query("SELECT count() FROM test.postgresql_replica") + print(result) # Just debug + + started_cluster.unpause_container('postgres1') + + check_tables_are_synchronized('postgresql_replica'); + + result = instance.query("SELECT count() FROM test.postgresql_replica") + print(result) # Just debug + + +def test_abrupt_server_restart_while_heavy_replication(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'); + + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') + create_materialized_table(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port) + + for query in queries: + cursor.execute(query) + print('query {}'.format(query)) + + instance.restart_clickhouse() + + result = instance.query("SELECT count() FROM test.postgresql_replica") + print(result) # Just debug + + check_tables_are_synchronized('postgresql_replica'); + + result = instance.query("SELECT count() FROM test.postgresql_replica") + print(result) # Just debug + + +def test_drop_table_immediately(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'); + instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)") + + instance.query('DROP TABLE IF EXISTS test.postgresql_replica') + create_materialized_table(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port) + instance.query('DROP TABLE test.postgresql_replica') + create_materialized_table(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port) + check_tables_are_synchronized('postgresql_replica'); + instance.query('DROP TABLE test.postgresql_replica') + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 4c8d82001f7ee3d7632463d10d9212701227b676 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Jul 2021 17:52:21 +0000 Subject: [PATCH 103/105] Fix checks --- .../integrations/materialized-postgresql.md | 2 +- src/Core/PostgreSQL/Connection.cpp | 2 +- .../PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 10 ---------- 3 files changed, 2 insertions(+), 12 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 30cc6a11c48..70f61c5b550 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -18,7 +18,7 @@ PRIMARY KEY key; - Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file. -- A table with engine `MaterializedPostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialize-postgresql.md#requirements)). +- A table with engine `MaterializedPostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialized-postgresql.md#requirements)). - Only database `Atomic` is allowed. diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp index d69856a392e..ad54bbe9dca 100644 --- a/src/Core/PostgreSQL/Connection.cpp +++ b/src/Core/PostgreSQL/Connection.cpp @@ -38,7 +38,7 @@ void Connection::execWithRetry(const std::function pqxx::connection & Connection::getRef() { connect(); - assert(connection != nulptr); + assert(connection != nullptr); return *connection; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index f80353f5586..b43e7656084 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -648,11 +648,6 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() LOG_ERROR(log, "Conversion error: {}", e.what()); return false; } - catch (const pqxx::statement_completion_unknown & e) - { - LOG_ERROR(log, "Unknown statement completion: {}", e.what()); - return false; - } catch (const pqxx::in_doubt_error & e) { LOG_ERROR(log, "PostgreSQL library has some doubts: {}", e.what()); @@ -663,11 +658,6 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() LOG_ERROR(log, "PostgreSQL library internal error: {}", e.what()); return false; } - catch (const pqxx::conversion_overrun & e) - { - LOG_ERROR(log, "PostgreSQL library conversion overflow: {}", e.what()); - return false; - } catch (...) { /// Since reading is done from a background task, it is important to catch any possible error From a0623ddb08e0ec9aea0509563d899acc6f531192 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 3 Jul 2021 13:35:11 +0000 Subject: [PATCH 104/105] Correct test --- .../test.py | 58 ++++++++++--------- 1 file changed, 30 insertions(+), 28 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index d13a7501b35..2d8689f31e8 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -72,8 +72,11 @@ def create_materialized_db(ip, port, instance.query(create_query) assert materialized_database in instance.query('SHOW DATABASES') -def drop_materialized_db(materialized_database='test_database'): - instance.query('DROP DATABASE {}'.format(materialized_database)) +def drop_materialized_db(materialized_database='test_database', sync=False): + if sync: + instance.query('DROP DATABASE IF EXISTS {} SYNC'.format(materialized_database)) + else: + instance.query('DROP DATABASE IF EXISTS {}'.format(materialized_database)) assert materialized_database not in instance.query('SHOW DATABASES') def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template): @@ -148,7 +151,7 @@ def started_cluster(): @pytest.mark.timeout(120) def test_load_and_sync_all_database_tables(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db(sync=True) conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -171,14 +174,12 @@ def test_load_and_sync_all_database_tables(started_cluster): result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''') assert(int(result) == NUM_TABLES) - - instance.query("DROP DATABASE test_database") - assert 'test_database' not in instance.query('SHOW DATABASES') + drop_materialized_db() @pytest.mark.timeout(120) def test_replicating_dml(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -217,13 +218,12 @@ def test_replicating_dml(started_cluster): for i in range(NUM_TABLES): cursor.execute('drop table postgresql_replica_{};'.format(i)) - instance.query("DROP DATABASE test_database") - assert 'test_database' not in instance.query('SHOW DATABASES') + drop_materialized_db() @pytest.mark.timeout(120) def test_different_data_types(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -303,13 +303,13 @@ def test_different_data_types(started_cluster): check_tables_are_synchronized('test_array_data_type'); result = instance.query('SELECT * FROM test_database.test_array_data_type ORDER BY key;') - instance.query("DROP DATABASE test_database") assert(result == expected) + drop_materialized_db() @pytest.mark.timeout(120) def test_load_and_sync_subset_of_database_tables(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db(sync=True) conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -355,14 +355,12 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): if i < int(NUM_TABLES/2): check_tables_are_synchronized(table_name); cursor.execute('drop table {};'.format(table_name)) - - instance.query("DROP DATABASE test_database") - assert 'test_database' not in instance.query('SHOW DATABASES') + drop_materialized_db() @pytest.mark.timeout(120) def test_changing_replica_identity_value(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -377,11 +375,12 @@ def test_changing_replica_identity_value(started_cluster): check_tables_are_synchronized('postgresql_replica'); cursor.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") check_tables_are_synchronized('postgresql_replica'); + drop_materialized_db() @pytest.mark.timeout(320) def test_clickhouse_restart(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -405,11 +404,12 @@ def test_clickhouse_restart(started_cluster): for i in range(NUM_TABLES): check_tables_are_synchronized('postgresql_replica_{}'.format(i)); + drop_materialized_db() @pytest.mark.timeout(120) def test_replica_identity_index(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -433,11 +433,12 @@ def test_replica_identity_index(started_cluster): cursor.execute('DELETE FROM postgresql_replica WHERE key2<75;') check_tables_are_synchronized('postgresql_replica', order_by='key1'); + drop_materialized_db() @pytest.mark.timeout(320) def test_table_schema_changes(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -493,7 +494,7 @@ def test_table_schema_changes(started_cluster): @pytest.mark.timeout(120) def test_many_concurrent_queries(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -573,7 +574,7 @@ def test_many_concurrent_queries(started_cluster): @pytest.mark.timeout(120) def test_single_transaction(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True, auto_commit=False) @@ -601,6 +602,7 @@ def test_single_transaction(started_cluster): def test_virtual_columns(started_cluster): + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -634,8 +636,8 @@ def test_virtual_columns(started_cluster): def test_multiple_databases(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database_1") - instance.query("DROP DATABASE IF EXISTS test_database_2") + drop_materialized_db('test_database_1') + drop_materialized_db('test_database_2') NUM_TABLES = 5 conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -692,7 +694,7 @@ def test_multiple_databases(started_cluster): @pytest.mark.timeout(320) def test_concurrent_transactions(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -737,7 +739,7 @@ def test_concurrent_transactions(started_cluster): @pytest.mark.timeout(320) def test_abrupt_connection_loss_while_heavy_replication(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -798,7 +800,7 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_drop_database_while_replication_startup_not_finished(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -817,7 +819,7 @@ def test_drop_database_while_replication_startup_not_finished(started_cluster): def test_restart_server_while_replication_startup_not_finished(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -839,7 +841,7 @@ def test_restart_server_while_replication_startup_not_finished(started_cluster): @pytest.mark.timeout(320) def test_abrupt_server_restart_while_heavy_replication(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) From b39888f199607d45b725f29668d48735d576507a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 4 Jul 2021 08:54:41 +0000 Subject: [PATCH 105/105] Do not allow inconsistency in publication --- .../DatabaseMaterializedPostgreSQL.cpp | 16 +++- .../PostgreSQLReplicationHandler.cpp | 75 ++++++++++++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- .../test.py | 26 +------ 4 files changed, 83 insertions(+), 36 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 4cfb5a4d137..256affc68c8 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -29,6 +29,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } DatabaseMaterializedPostgreSQL::DatabaseMaterializedPostgreSQL( @@ -63,7 +64,19 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() settings->materialized_postgresql_tables_list.value); postgres::Connection connection(connection_info); - std::unordered_set tables_to_replicate = replication_handler->fetchRequiredTables(connection.getRef()); + NameSet tables_to_replicate; + try + { + tables_to_replicate = replication_handler->fetchRequiredTables(connection); + } + catch (...) + { + LOG_ERROR(log, "Unable to load replicated tables list"); + throw; + } + + if (tables_to_replicate.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of tables to replicate"); for (const auto & table_name : tables_to_replicate) { @@ -156,6 +169,7 @@ void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const void DatabaseMaterializedPostgreSQL::shutdown() { stopReplication(); + DatabaseAtomic::shutdown(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 1d8ab04cfec..67026d345eb 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -20,6 +20,7 @@ static const auto BACKOFF_TRESHOLD_MS = 10000; namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( @@ -434,25 +435,75 @@ void PostgreSQLReplicationHandler::shutdownFinal() /// Used by MaterializedPostgreSQL database engine. -NameSet PostgreSQLReplicationHandler::fetchRequiredTables(pqxx::connection & connection_) +NameSet PostgreSQLReplicationHandler::fetchRequiredTables(postgres::Connection & connection_) { - pqxx::work tx(connection_); - bool publication_exists = isPublicationExist(tx); + pqxx::work tx(connection_.getRef()); + bool publication_exists_before_startup = isPublicationExist(tx); NameSet result_tables; - if (tables_list.empty() && !publication_exists) + Strings expected_tables; + if (!tables_list.empty()) { - /// Fetch all tables list from database. Publication does not exist yet, which means - /// that no replication took place. Publication will be created in - /// startSynchronization method. - result_tables = fetchPostgreSQLTablesList(tx); + splitInto<','>(expected_tables, tables_list); + if (expected_tables.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse tables list: {}", tables_list); + for (auto & table_name : expected_tables) + boost::trim(table_name); + } + + if (publication_exists_before_startup) + { + if (tables_list.empty()) + { + /// There is no tables list, but publication already exists, then the expected behaviour + /// is to replicate the whole database. But it could be a server restart, so we can't drop it. + LOG_WARNING(log, + "Publication {} already exists and tables list is empty. Assuming publication is correct", + publication_name); + + result_tables = fetchPostgreSQLTablesList(tx); + } + /// Check tables list from publication is the same as expected tables list. + /// If not - drop publication and return expected tables list. + else + { + result_tables = fetchTablesFromPublication(tx); + NameSet diff; + std::set_symmetric_difference(expected_tables.begin(), expected_tables.end(), + result_tables.begin(), result_tables.end(), + std::inserter(diff, diff.begin())); + if (!diff.empty()) + { + String diff_tables; + for (const auto & table_name : diff) + { + if (!diff_tables.empty()) + diff_tables += ", "; + diff_tables += table_name; + } + + LOG_WARNING(log, + "Publication {} already exists, but specified tables list differs from publication tables list in tables: {}", + publication_name, diff_tables); + + connection->execWithRetry([&](pqxx::nontransaction & tx_){ dropPublication(tx_); }); + } + } } else { - if (!publication_exists) - createPublicationIfNeeded(tx, /* create_without_check = */ true); - - result_tables = fetchTablesFromPublication(tx); + if (!tables_list.empty()) + { + tx.commit(); + return NameSet(expected_tables.begin(), expected_tables.end()); + } + else + { + /// Fetch all tables list from database. Publication does not exist yet, which means + /// that no replication took place. Publication will be created in + /// startSynchronization method. + result_tables = fetchPostgreSQLTablesList(tx); + } } tx.commit(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 6ae9ec31626..4b6321338b8 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -42,7 +42,7 @@ public: void addStorage(const std::string & table_name, StorageMaterializedPostgreSQL * storage); /// Fetch list of tables which are going to be replicated. Used for database engine. - NameSet fetchRequiredTables(pqxx::connection & connection_); + NameSet fetchRequiredTables(postgres::Connection & connection_); /// Start replication setup immediately. void startSynchronization(bool throw_on_error); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 2d8689f31e8..3a6bb29585e 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -72,11 +72,8 @@ def create_materialized_db(ip, port, instance.query(create_query) assert materialized_database in instance.query('SHOW DATABASES') -def drop_materialized_db(materialized_database='test_database', sync=False): - if sync: - instance.query('DROP DATABASE IF EXISTS {} SYNC'.format(materialized_database)) - else: - instance.query('DROP DATABASE IF EXISTS {}'.format(materialized_database)) +def drop_materialized_db(materialized_database='test_database'): + instance.query('DROP DATABASE IF EXISTS {}'.format(materialized_database)) assert materialized_database not in instance.query('SHOW DATABASES') def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template): @@ -108,7 +105,6 @@ queries = [ ] -@pytest.mark.timeout(30) def assert_nested_table_is_created(table_name, materialized_database='test_database'): database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database)) while table_name not in database_tables: @@ -117,7 +113,6 @@ def assert_nested_table_is_created(table_name, materialized_database='test_datab assert(table_name in database_tables) -@pytest.mark.timeout(30) def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'): assert_nested_table_is_created(table_name, materialized_database) @@ -149,9 +144,8 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.timeout(120) def test_load_and_sync_all_database_tables(started_cluster): - drop_materialized_db(sync=True) + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -177,7 +171,6 @@ def test_load_and_sync_all_database_tables(started_cluster): drop_materialized_db() -@pytest.mark.timeout(120) def test_replicating_dml(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -221,7 +214,6 @@ def test_replicating_dml(started_cluster): drop_materialized_db() -@pytest.mark.timeout(120) def test_different_data_types(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -307,9 +299,8 @@ def test_different_data_types(started_cluster): drop_materialized_db() -@pytest.mark.timeout(120) def test_load_and_sync_subset_of_database_tables(started_cluster): - drop_materialized_db(sync=True) + drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) @@ -358,7 +349,6 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): drop_materialized_db() -@pytest.mark.timeout(120) def test_changing_replica_identity_value(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -378,7 +368,6 @@ def test_changing_replica_identity_value(started_cluster): drop_materialized_db() -@pytest.mark.timeout(320) def test_clickhouse_restart(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -407,7 +396,6 @@ def test_clickhouse_restart(started_cluster): drop_materialized_db() -@pytest.mark.timeout(120) def test_replica_identity_index(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -436,7 +424,6 @@ def test_replica_identity_index(started_cluster): drop_materialized_db() -@pytest.mark.timeout(320) def test_table_schema_changes(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -492,7 +479,6 @@ def test_table_schema_changes(started_cluster): instance.query("DROP DATABASE test_database") -@pytest.mark.timeout(120) def test_many_concurrent_queries(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -572,7 +558,6 @@ def test_many_concurrent_queries(started_cluster): drop_materialized_db() -@pytest.mark.timeout(120) def test_single_transaction(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -692,7 +677,6 @@ def test_multiple_databases(started_cluster): drop_materialized_db('test_database_2') -@pytest.mark.timeout(320) def test_concurrent_transactions(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -737,7 +721,6 @@ def test_concurrent_transactions(started_cluster): drop_materialized_db() -@pytest.mark.timeout(320) def test_abrupt_connection_loss_while_heavy_replication(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip, @@ -839,7 +822,6 @@ def test_restart_server_while_replication_startup_not_finished(started_cluster): drop_materialized_db() -@pytest.mark.timeout(320) def test_abrupt_server_restart_while_heavy_replication(started_cluster): drop_materialized_db() conn = get_postgres_conn(ip=started_cluster.postgres_ip,