mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Move connection-related filed to Core/PostgreSQL
This commit is contained in:
parent
9fa21a6272
commit
8586565ce9
@ -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()
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include "PostgreSQLConnection.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
@ -6,6 +6,7 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -14,7 +15,6 @@ namespace ErrorCodes
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -10,8 +10,8 @@
|
||||
#include <Core/ExternalResultDescription.h>
|
||||
#include <Core/Field.h>
|
||||
#include <pqxx/pqxx>
|
||||
#include <Storages/PostgreSQL/insertPostgreSQLValue.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Core/PostgreSQL/insertPostgreSQLValue.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLConnection.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -39,7 +39,7 @@
|
||||
#include <Databases/PostgreSQL/DatabasePostgreSQL.h> // Y_IGNORE
|
||||
#include <Databases/PostgreSQL/DatabaseMaterializePostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializePostgreSQLSettings.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnectionPool.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLConnectionPool.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Storages/PostgreSQL/StorageMaterializePostgreSQL.h>
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
@ -16,7 +16,6 @@
|
||||
#include <Poco/File.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnectionPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,7 +5,7 @@
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
|
||||
|
@ -11,7 +11,7 @@
|
||||
#include <Core/Block.h>
|
||||
#include <common/LocalDateTime.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
#include <pqxx/pqxx>
|
||||
|
||||
|
||||
|
@ -5,9 +5,9 @@
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include "PostgreSQLConnection.h"
|
||||
#include "MaterializePostgreSQLMetadata.h"
|
||||
#include "insertPostgreSQLValue.h"
|
||||
#include <Core/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Core/PostgreSQL/insertPostgreSQLValue.h>
|
||||
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/Names.h>
|
||||
|
@ -1,39 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include "PostgreSQLConnectionPool.h"
|
||||
#include <mutex>
|
||||
|
||||
|
||||
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<size_t, PostgreSQLConnectionPoolPtr>;
|
||||
|
||||
ReplicasByPriority replicas;
|
||||
size_t num_retries;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
using PostgreSQLReplicaConnectionPtr = std::shared_ptr<PostgreSQLReplicaConnection>;
|
||||
|
||||
}
|
@ -5,9 +5,11 @@
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include "PostgreSQLConnection.h"
|
||||
|
||||
#include "MaterializePostgreSQLConsumer.h"
|
||||
#include "MaterializePostgreSQLMetadata.h"
|
||||
|
||||
#include <Core/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
|
||||
|
||||
|
@ -22,7 +22,7 @@
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Storages/ReadFinalForExternalReplicaStorage.h>
|
||||
#include "PostgreSQLConnectionPool.h"
|
||||
#include <Core/PostgreSQL/PostgreSQLConnectionPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include "StoragePostgreSQL.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <DataStreams/PostgreSQLBlockInputStream.h>
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
#include <pqxx/pqxx>
|
||||
|
||||
|
||||
|
@ -12,7 +12,6 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include "registerTableFunctions.h"
|
||||
#include <Storages/PostgreSQL/PostgreSQLConnection.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
Loading…
Reference in New Issue
Block a user