mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #33187 from kssenii/materialized-postgresql-fix-cleanup
materialized postgresql make sure temporary replication slots are deleted
This commit is contained in:
commit
0c41b46e75
@ -20,6 +20,7 @@ namespace DB
|
||||
|
||||
static const auto RESCHEDULE_MS = 1000;
|
||||
static const auto BACKOFF_TRESHOLD_MS = 10000;
|
||||
static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
@ -28,6 +29,30 @@ namespace ErrorCodes
|
||||
extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR;
|
||||
}
|
||||
|
||||
class TemporaryReplicationSlot
|
||||
{
|
||||
public:
|
||||
TemporaryReplicationSlot(
|
||||
PostgreSQLReplicationHandler * handler_,
|
||||
std::shared_ptr<pqxx::nontransaction> tx_,
|
||||
String & start_lsn,
|
||||
String & snapshot_name)
|
||||
: handler(handler_), tx(tx_)
|
||||
{
|
||||
handler->createReplicationSlot(*tx, start_lsn, snapshot_name, /* temporary */true);
|
||||
}
|
||||
|
||||
~TemporaryReplicationSlot()
|
||||
{
|
||||
handler->dropReplicationSlot(*tx, /* temporary */true);
|
||||
}
|
||||
|
||||
private:
|
||||
PostgreSQLReplicationHandler * handler;
|
||||
std::shared_ptr<pqxx::nontransaction> tx;
|
||||
};
|
||||
|
||||
|
||||
PostgreSQLReplicationHandler::PostgreSQLReplicationHandler(
|
||||
const String & replication_identifier,
|
||||
const String & postgres_database_,
|
||||
@ -69,6 +94,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler(
|
||||
|
||||
startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); });
|
||||
consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); });
|
||||
cleanup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ cleanupFunc(); });
|
||||
}
|
||||
|
||||
|
||||
@ -148,6 +174,7 @@ void PostgreSQLReplicationHandler::shutdown()
|
||||
stop_synchronization.store(true);
|
||||
startup_task->deactivate();
|
||||
consumer_task->deactivate();
|
||||
cleanup_task->deactivate();
|
||||
}
|
||||
|
||||
|
||||
@ -268,6 +295,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error)
|
||||
(is_materialized_postgresql_database ? postgres_database : postgres_database + '.' + tables_list));
|
||||
|
||||
consumer_task->activateAndSchedule();
|
||||
cleanup_task->activateAndSchedule();
|
||||
|
||||
/// Do not rely anymore on saved storage pointers.
|
||||
materialized_storages.clear();
|
||||
@ -278,6 +306,7 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ
|
||||
{
|
||||
postgres::Connection connection(connection_info);
|
||||
pqxx::nontransaction tx(connection.getRef());
|
||||
|
||||
auto [postgres_table_schema, postgres_table_name] = getSchemaAndTableName(table_name);
|
||||
auto table_structure = std::make_unique<PostgreSQLTableStructure>(fetchPostgreSQLTableStructure(tx, postgres_table_name, postgres_table_schema, true, true, true));
|
||||
|
||||
@ -330,6 +359,21 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection &
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::cleanupFunc()
|
||||
{
|
||||
/// It is very important to make sure temporary replication slots are removed!
|
||||
/// So just in case every 30 minutes check if one still exists.
|
||||
postgres::Connection connection(connection_info);
|
||||
String last_committed_lsn;
|
||||
connection.execWithRetry([&](pqxx::nontransaction & tx)
|
||||
{
|
||||
if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true))
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
});
|
||||
cleanup_task->scheduleAfter(CLEANUP_RESCHEDULE_MS);
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::consumerFunc()
|
||||
{
|
||||
std::vector<std::pair<Int32, String>> skipped_tables;
|
||||
@ -774,10 +818,12 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost
|
||||
StoragePtr nested_storage;
|
||||
|
||||
{
|
||||
pqxx::nontransaction tx(replication_connection.getRef());
|
||||
if (isReplicationSlotExist(tx, start_lsn, /* temporary */true))
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true);
|
||||
auto tx = std::make_shared<pqxx::nontransaction>(replication_connection.getRef());
|
||||
|
||||
if (isReplicationSlotExist(*tx, start_lsn, /* temporary */true))
|
||||
dropReplicationSlot(*tx, /* temporary */true);
|
||||
|
||||
TemporaryReplicationSlot temporary_slot(this, tx, start_lsn, snapshot_name);
|
||||
|
||||
/// Protect against deadlock.
|
||||
auto nested = DatabaseCatalog::instance().tryGetTable(materialized_storage->getNestedStorageID(), materialized_storage->getNestedTableContext());
|
||||
@ -848,81 +894,81 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector<std::pai
|
||||
try
|
||||
{
|
||||
postgres::Connection replication_connection(connection_info, /* replication */true);
|
||||
pqxx::nontransaction tx(replication_connection.getRef());
|
||||
auto tx = std::make_shared<pqxx::nontransaction>(replication_connection.getRef());
|
||||
|
||||
String snapshot_name, start_lsn;
|
||||
|
||||
if (isReplicationSlotExist(tx, start_lsn, /* temporary */true))
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
|
||||
createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true);
|
||||
postgres::Connection tmp_connection(connection_info);
|
||||
|
||||
for (const auto & [relation_id, table_name] : relation_data)
|
||||
{
|
||||
auto storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context);
|
||||
auto * materialized_storage = storage->as <StorageMaterializedPostgreSQL>();
|
||||
auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout);
|
||||
String snapshot_name, start_lsn;
|
||||
if (isReplicationSlotExist(*tx, start_lsn, /* temporary */true))
|
||||
dropReplicationSlot(*tx, /* temporary */true);
|
||||
|
||||
/// If for some reason this temporary table already exists - also drop it.
|
||||
auto temp_materialized_storage = materialized_storage->createTemporary();
|
||||
TemporaryReplicationSlot temporary_slot(this, tx, start_lsn, snapshot_name);
|
||||
postgres::Connection tmp_connection(connection_info);
|
||||
|
||||
/// This snapshot is valid up to the end of the transaction, which exported it.
|
||||
StoragePtr temp_nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, table_name,
|
||||
temp_materialized_storage->as <StorageMaterializedPostgreSQL>());
|
||||
|
||||
auto table_id = materialized_storage->getNestedStorageID();
|
||||
auto temp_table_id = temp_nested_storage->getStorageID();
|
||||
|
||||
LOG_DEBUG(log, "Starting background update of table {} ({} with {})",
|
||||
table_name, table_id.getNameForLogs(), temp_table_id.getNameForLogs());
|
||||
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>();
|
||||
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 <StorageMaterializedPostgreSQL>();
|
||||
auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
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
|
||||
{
|
||||
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(tmp_connection, snapshot_name, table_name,
|
||||
temp_materialized_storage->as <StorageMaterializedPostgreSQL>());
|
||||
|
||||
auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name, temp_table_id.uuid), nested_context);
|
||||
materialized_storage->set(nested_storage);
|
||||
auto table_id = materialized_storage->getNestedStorageID();
|
||||
auto temp_table_id = temp_nested_storage->getStorageID();
|
||||
|
||||
auto nested_sample_block = nested_storage->getInMemoryMetadataPtr()->getSampleBlock();
|
||||
auto materialized_sample_block = materialized_storage->getInMemoryMetadataPtr()->getSampleBlock();
|
||||
assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background");
|
||||
LOG_DEBUG(log, "Starting background update of table {} ({} with {})",
|
||||
table_name, table_id.getNameForLogs(), temp_table_id.getNameForLogs());
|
||||
|
||||
LOG_INFO(log, "Updated table {}. New structure: {}",
|
||||
nested_storage->getStorageID().getNameForLogs(), nested_sample_block.dumpStructure());
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>();
|
||||
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;
|
||||
|
||||
/// 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_context = materialized_storage->getNestedTableContext();
|
||||
|
||||
auto table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context);
|
||||
auto drop_table_id = table_to_drop->getStorageID();
|
||||
try
|
||||
{
|
||||
InterpreterRenameQuery(ast_rename, nested_context).execute();
|
||||
|
||||
if (drop_table_id == nested_storage->getStorageID())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot drop table because is has the same uuid as new table: {}", drop_table_id.getNameForLogs());
|
||||
auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name, temp_table_id.uuid), nested_context);
|
||||
materialized_storage->set(nested_storage);
|
||||
|
||||
LOG_DEBUG(log, "Dropping table {}", drop_table_id.getNameForLogs());
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, drop_table_id, true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
auto nested_sample_block = nested_storage->getInMemoryMetadataPtr()->getSampleBlock();
|
||||
auto materialized_sample_block = materialized_storage->getInMemoryMetadataPtr()->getSampleBlock();
|
||||
assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background");
|
||||
|
||||
LOG_INFO(log, "Updated table {}. New structure: {}",
|
||||
nested_storage->getStorageID().getNameForLogs(), nested_sample_block.dumpStructure());
|
||||
|
||||
/// 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 table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context);
|
||||
auto drop_table_id = table_to_drop->getStorageID();
|
||||
|
||||
if (drop_table_id == nested_storage->getStorageID())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot drop table because is has the same uuid as new table: {}", drop_table_id.getNameForLogs());
|
||||
|
||||
LOG_DEBUG(log, "Dropping table {}", drop_table_id.getNameForLogs());
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, drop_table_id, true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
tx.commit();
|
||||
tx->commit();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -15,6 +15,8 @@ struct SettingChange;
|
||||
|
||||
class PostgreSQLReplicationHandler
|
||||
{
|
||||
friend class TemporaryReplicationSlot;
|
||||
|
||||
public:
|
||||
PostgreSQLReplicationHandler(
|
||||
const String & replication_identifier,
|
||||
@ -52,6 +54,8 @@ public:
|
||||
|
||||
void setSetting(const SettingChange & setting);
|
||||
|
||||
void cleanupFunc();
|
||||
|
||||
private:
|
||||
using MaterializedStorages = std::unordered_map<String, StorageMaterializedPostgreSQL *>;
|
||||
|
||||
@ -133,7 +137,9 @@ private:
|
||||
/// Replication consumer. Manages decoding of replication stream and syncing into tables.
|
||||
std::shared_ptr<MaterializedPostgreSQLConsumer> consumer;
|
||||
|
||||
BackgroundSchedulePool::TaskHolder startup_task, consumer_task;
|
||||
BackgroundSchedulePool::TaskHolder startup_task;
|
||||
BackgroundSchedulePool::TaskHolder consumer_task;
|
||||
BackgroundSchedulePool::TaskHolder cleanup_task;
|
||||
|
||||
std::atomic<bool> stop_synchronization = false;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user