mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
More robust
This commit is contained in:
parent
159de92197
commit
db998c3f6c
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
}
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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))
|
||||
|
Loading…
Reference in New Issue
Block a user