mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
dbms: Server: further simplification of memory ordering [#METR-14410]
This commit is contained in:
parent
c5b61652ff
commit
ca1c0756c2
@ -22,6 +22,7 @@ class RemoteBlockInputStream : public IProfilingBlockInputStream
|
||||
private:
|
||||
void init(const Settings * settings_)
|
||||
{
|
||||
established.store(false, std::memory_order_seq_cst);
|
||||
sent_query.store(false, std::memory_order_seq_cst);
|
||||
finished.store(false, std::memory_order_seq_cst);
|
||||
got_exception_from_replica.store(false, std::memory_order_seq_cst);
|
||||
@ -106,7 +107,7 @@ public:
|
||||
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
|
||||
* эти соединения не остались висеть в рассихронизированном состоянии.
|
||||
*/
|
||||
if (established || isQueryInProgress())
|
||||
if (established.load(std::memory_order_seq_cst) || isQueryInProgress())
|
||||
parallel_replicas->disconnect();
|
||||
}
|
||||
|
||||
@ -144,10 +145,14 @@ protected:
|
||||
if (!sent_query.load(std::memory_order_seq_cst))
|
||||
{
|
||||
createParallelReplicas();
|
||||
established = true;
|
||||
|
||||
established.store(true, std::memory_order_seq_cst);
|
||||
|
||||
parallel_replicas->sendQuery(query, "", stage, true);
|
||||
established = false;
|
||||
|
||||
established.store(false, std::memory_order_seq_cst);
|
||||
sent_query.store(true, std::memory_order_seq_cst);
|
||||
|
||||
sendExternalTables();
|
||||
}
|
||||
|
||||
@ -319,7 +324,7 @@ private:
|
||||
Context context;
|
||||
|
||||
/// Установили соединения с репликами, но ещё не отправили запрос.
|
||||
volatile bool established = false;
|
||||
std::atomic<bool> established;
|
||||
|
||||
/// Отправили запрос (это делается перед получением первого блока).
|
||||
std::atomic<bool> sent_query;
|
||||
|
Loading…
Reference in New Issue
Block a user