More robust

This commit is contained in:
kssenii 2021-06-27 16:15:28 +00:00
parent 159de92197
commit db998c3f6c
5 changed files with 117 additions and 48 deletions

View File

@ -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<void(pqxx::nontransaction &)> & 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;
}

View File

@ -13,7 +13,9 @@ using ConnectionPtr = std::unique_ptr<pqxx::connection>;
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<void(pqxx::nontransaction &)> & exec);
pqxx::connection & getRef();
@ -24,6 +26,8 @@ public:
private:
ConnectionPtr connection;
ConnectionInfo connection_info;
bool replication;
size_t num_tries;
};
}

View File

@ -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<pqxx::nontransaction> 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_ptr<pqxx::nontransact
assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "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<pqxx::nontransa
if (!result.empty())
return result[0][0].as<std::string>();
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())
{

View File

@ -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)
{

View File

@ -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))