mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Better
This commit is contained in:
parent
51cb89973d
commit
3903d59d30
@ -28,14 +28,13 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
PostgreSQLBlockInputStream::PostgreSQLBlockInputStream(
|
||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
||||
WrappedPostgreSQLConnection connection_,
|
||||
const std::string & query_str_,
|
||||
const Block & sample_block,
|
||||
const UInt64 max_block_size_)
|
||||
: query_str(query_str_)
|
||||
, max_block_size(max_block_size_)
|
||||
, connection_pool(connection_pool_)
|
||||
, connection(connection_pool->get())
|
||||
, connection(connection_)
|
||||
{
|
||||
description.init(sample_block);
|
||||
for (const auto idx : ext::range(0, description.sample_block.columns()))
|
||||
@ -112,9 +111,6 @@ void PostgreSQLBlockInputStream::readSuffix()
|
||||
stream->complete();
|
||||
tx->commit();
|
||||
}
|
||||
|
||||
if (connection->is_open())
|
||||
connection_pool->put(connection);
|
||||
}
|
||||
|
||||
|
||||
|
@ -19,7 +19,7 @@ class PostgreSQLBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
PostgreSQLBlockInputStream(
|
||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
||||
WrappedPostgreSQLConnection connection_,
|
||||
const std::string & query_str,
|
||||
const Block & sample_block,
|
||||
const UInt64 max_block_size_);
|
||||
@ -46,8 +46,7 @@ private:
|
||||
const UInt64 max_block_size;
|
||||
ExternalResultDescription description;
|
||||
|
||||
PostgreSQLConnectionPoolPtr connection_pool;
|
||||
PostgreSQLConnection::ConnectionPtr connection;
|
||||
WrappedPostgreSQLConnection connection;
|
||||
std::unique_ptr<pqxx::read_transaction> tx;
|
||||
std::unique_ptr<pqxx::stream_from> stream;
|
||||
|
||||
|
@ -96,7 +96,6 @@ std::unordered_set<std::string> DatabasePostgreSQL::fetchTablesList() const
|
||||
for (auto table_name : tx.stream<std::string>(query))
|
||||
tables.insert(std::get<0>(table_name));
|
||||
|
||||
connection_pool->put(connection);
|
||||
return tables;
|
||||
}
|
||||
|
||||
@ -132,7 +131,6 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const
|
||||
throw;
|
||||
}
|
||||
|
||||
connection_pool->put(connection);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -167,7 +165,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte
|
||||
return StoragePtr{};
|
||||
|
||||
auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls;
|
||||
auto columns = fetchPostgreSQLTableStructure(connection_pool->get(), table_name, use_nulls);
|
||||
auto columns = fetchPostgreSQLTableStructure(connection_pool->get(), doubleQuoteString(table_name), use_nulls);
|
||||
|
||||
if (!columns)
|
||||
return StoragePtr{};
|
||||
|
@ -94,7 +94,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl
|
||||
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||
std::shared_ptr<pqxx::connection> connection, const String & postgres_table_name, bool use_nulls)
|
||||
WrappedPostgreSQLConnection connection, const String & postgres_table_name, bool use_nulls)
|
||||
{
|
||||
auto columns = NamesAndTypesList();
|
||||
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
{
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||
std::shared_ptr<pqxx::connection> connection, const String & postgres_table_name, bool use_nulls);
|
||||
WrappedPostgreSQLConnection connection, const String & postgres_table_name, bool use_nulls);
|
||||
|
||||
}
|
||||
|
||||
|
@ -20,17 +20,6 @@ PostgreSQLConnection::PostgreSQLConnection(
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnection::PostgreSQLConnection(
|
||||
ConnectionPtr connection_,
|
||||
const String & connection_str_,
|
||||
const String & address_)
|
||||
: connection(std::move(connection_))
|
||||
, connection_str(connection_str_)
|
||||
, address(address_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnection::PostgreSQLConnection(const PostgreSQLConnection & other)
|
||||
: connection_str(other.connection_str)
|
||||
, address(other.address)
|
||||
@ -40,31 +29,31 @@ PostgreSQLConnection::PostgreSQLConnection(const PostgreSQLConnection & other)
|
||||
|
||||
PostgreSQLConnection::ConnectionPtr PostgreSQLConnection::get()
|
||||
{
|
||||
connect();
|
||||
connectIfNeeded();
|
||||
return connection;
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnection::ConnectionPtr PostgreSQLConnection::tryGet()
|
||||
{
|
||||
if (tryConnect())
|
||||
if (tryConnectIfNeeded())
|
||||
return connection;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLConnection::connect()
|
||||
void PostgreSQLConnection::connectIfNeeded()
|
||||
{
|
||||
if (!connection || !connection->is_open())
|
||||
connection = std::make_shared<pqxx::connection>(connection_str);
|
||||
}
|
||||
|
||||
|
||||
bool PostgreSQLConnection::tryConnect()
|
||||
bool PostgreSQLConnection::tryConnectIfNeeded()
|
||||
{
|
||||
try
|
||||
{
|
||||
connect();
|
||||
connectIfNeeded();
|
||||
}
|
||||
catch (const pqxx::broken_connection & pqxx_error)
|
||||
{
|
||||
|
@ -12,44 +12,65 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class PostgreSQLConnection
|
||||
{
|
||||
|
||||
using ConnectionPtr = std::shared_ptr<pqxx::connection>;
|
||||
|
||||
public:
|
||||
using ConnectionPtr = std::shared_ptr<pqxx::connection>;
|
||||
|
||||
PostgreSQLConnection(
|
||||
const String & connection_str_,
|
||||
const String & address_);
|
||||
|
||||
PostgreSQLConnection(
|
||||
ConnectionPtr connection_,
|
||||
const String & connection_str_,
|
||||
const String & address_);
|
||||
|
||||
PostgreSQLConnection(const PostgreSQLConnection & other);
|
||||
|
||||
const std::string & getAddress() { return address; }
|
||||
|
||||
ConnectionPtr get();
|
||||
|
||||
ConnectionPtr tryGet();
|
||||
|
||||
bool connected() { return tryConnect(); }
|
||||
bool isConnected() { return tryConnectIfNeeded(); }
|
||||
|
||||
bool available() { return ref_count.load() == 0; }
|
||||
|
||||
void incrementRef() { ref_count++; }
|
||||
|
||||
void decrementRef() { ref_count--; }
|
||||
|
||||
private:
|
||||
void connect();
|
||||
void connectIfNeeded();
|
||||
|
||||
bool tryConnect();
|
||||
bool tryConnectIfNeeded();
|
||||
|
||||
const std::string & getAddress() { return address; }
|
||||
|
||||
ConnectionPtr connection;
|
||||
std::string connection_str;
|
||||
std::string address;
|
||||
std::string connection_str, address;
|
||||
std::atomic<uint8_t> ref_count;
|
||||
};
|
||||
|
||||
using PostgreSQLConnectionPtr = std::shared_ptr<PostgreSQLConnection>;
|
||||
|
||||
|
||||
class WrappedPostgreSQLConnection
|
||||
{
|
||||
|
||||
public:
|
||||
WrappedPostgreSQLConnection(PostgreSQLConnectionPtr connection_) : connection(connection_) { connection->incrementRef(); }
|
||||
|
||||
WrappedPostgreSQLConnection(const WrappedPostgreSQLConnection & other) : connection(other.connection) {}
|
||||
|
||||
~WrappedPostgreSQLConnection() { connection->decrementRef(); }
|
||||
|
||||
pqxx::connection & operator*() const { return *connection->get(); }
|
||||
|
||||
pqxx::connection * operator->() const { return connection->get().get(); }
|
||||
|
||||
bool isConnected() { return connection->isConnected(); }
|
||||
|
||||
private:
|
||||
PostgreSQLConnectionPtr connection;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -14,22 +14,18 @@ namespace DB
|
||||
|
||||
PostgreSQLConnectionPool::PostgreSQLConnectionPool(
|
||||
std::string dbname, std::string host, UInt16 port, std::string user, std::string password)
|
||||
: pool(POSTGRESQL_POOL_DEFAULT_SIZE)
|
||||
{
|
||||
address = host + ':' + std::to_string(port);
|
||||
connection_str = formatConnectionString(std::move(dbname), std::move(host), port, std::move(user), std::move(password));
|
||||
|
||||
/// No connection is made, just fill pool with non-connected connection objects.
|
||||
for (size_t i = 0; i < POSTGRESQL_POOL_DEFAULT_SIZE; ++i)
|
||||
pool.push(std::make_shared<PostgreSQLConnection>(connection_str, address));
|
||||
initialize();
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnectionPool::PostgreSQLConnectionPool(const PostgreSQLConnectionPool & other)
|
||||
: connection_str(other.connection_str)
|
||||
, address(other.address)
|
||||
, pool(POSTGRESQL_POOL_DEFAULT_SIZE)
|
||||
{
|
||||
initialize();
|
||||
}
|
||||
|
||||
|
||||
@ -46,48 +42,34 @@ std::string PostgreSQLConnectionPool::formatConnectionString(
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnection::ConnectionPtr PostgreSQLConnectionPool::get()
|
||||
void PostgreSQLConnectionPool::initialize()
|
||||
{
|
||||
PostgreSQLConnectionPtr connection = popConnection();
|
||||
return connection->get();
|
||||
/// No connection is made, just fill pool with non-connected connection objects.
|
||||
for (size_t i = 0; i < POSTGRESQL_POOL_DEFAULT_SIZE; ++i)
|
||||
pool.emplace_back(std::make_shared<PostgreSQLConnection>(connection_str, address));
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnection::ConnectionPtr PostgreSQLConnectionPool::tryGet()
|
||||
|
||||
WrappedPostgreSQLConnection PostgreSQLConnectionPool::get()
|
||||
{
|
||||
PostgreSQLConnectionPtr connection = popConnection();
|
||||
return connection->tryGet();
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
for (const auto & connection : pool)
|
||||
{
|
||||
if (connection->available())
|
||||
return WrappedPostgreSQLConnection(connection);
|
||||
}
|
||||
|
||||
auto connection = std::make_shared<PostgreSQLConnection>(connection_str, address);
|
||||
return WrappedPostgreSQLConnection(connection);
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnectionPtr PostgreSQLConnectionPool::popConnection()
|
||||
bool PostgreSQLConnectionPool::isConnected()
|
||||
{
|
||||
PostgreSQLConnectionPtr connection;
|
||||
if (pool.tryPop(connection, POSTGRESQL_POOL_WAIT_POP_PUSH_MS))
|
||||
return connection;
|
||||
|
||||
return std::make_shared<PostgreSQLConnection>(connection_str, address);
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLConnectionPool::put(PostgreSQLConnection::ConnectionPtr connection)
|
||||
{
|
||||
pushConnection(std::make_shared<PostgreSQLConnection>(connection, connection_str, address));
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLConnectionPool::pushConnection(PostgreSQLConnectionPtr connection)
|
||||
{
|
||||
pool.tryPush(connection, POSTGRESQL_POOL_WAIT_POP_PUSH_MS);
|
||||
}
|
||||
|
||||
|
||||
bool PostgreSQLConnectionPool::connected()
|
||||
{
|
||||
PostgreSQLConnectionPtr connection = popConnection();
|
||||
bool result = connection->connected();
|
||||
pushConnection(connection);
|
||||
return result;
|
||||
auto connection = get();
|
||||
return connection.isConnected();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -7,13 +7,15 @@
|
||||
#if USE_LIBPQXX
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include "PostgreSQLConnection.h"
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class PostgreSQLReplicaConnection;
|
||||
|
||||
class PostgreSQLConnectionPool
|
||||
{
|
||||
friend class PostgreSQLReplicaConnection;
|
||||
|
||||
public:
|
||||
|
||||
@ -23,36 +25,25 @@ public:
|
||||
|
||||
PostgreSQLConnectionPool operator =(const PostgreSQLConnectionPool &) = delete;
|
||||
|
||||
/// Will throw if unable to setup connection.
|
||||
PostgreSQLConnection::ConnectionPtr get();
|
||||
WrappedPostgreSQLConnection get();
|
||||
|
||||
/// Will return nullptr if connection was not established.
|
||||
PostgreSQLConnection::ConnectionPtr tryGet();
|
||||
|
||||
void put(PostgreSQLConnection::ConnectionPtr connection);
|
||||
|
||||
bool connected();
|
||||
|
||||
std::string & conn_str() { return connection_str; }
|
||||
protected:
|
||||
bool isConnected();
|
||||
|
||||
private:
|
||||
static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16;
|
||||
static constexpr inline auto POSTGRESQL_POOL_WAIT_POP_PUSH_MS = 100;
|
||||
|
||||
using Pool = ConcurrentBoundedQueue<PostgreSQLConnectionPtr>;
|
||||
using Pool = std::vector<PostgreSQLConnectionPtr>;
|
||||
|
||||
static std::string formatConnectionString(
|
||||
std::string dbname, std::string host, UInt16 port, std::string user, std::string password);
|
||||
|
||||
/// Try get connection from connection pool with timeout.
|
||||
/// If pool is empty after timeout, make a new connection.
|
||||
PostgreSQLConnectionPtr popConnection();
|
||||
|
||||
/// Put connection object back into pool.
|
||||
void pushConnection(PostgreSQLConnectionPtr connection);
|
||||
void initialize();
|
||||
|
||||
std::string connection_str, address;
|
||||
Pool pool;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
using PostgreSQLConnectionPoolPtr = std::shared_ptr<PostgreSQLConnectionPool>;
|
||||
|
@ -59,14 +59,14 @@ PostgreSQLReplicaConnection::PostgreSQLReplicaConnection(const PostgreSQLReplica
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLConnectionPoolPtr PostgreSQLReplicaConnection::get()
|
||||
WrappedPostgreSQLConnection PostgreSQLReplicaConnection::get()
|
||||
{
|
||||
for (size_t i = 0; i < num_retries; ++i)
|
||||
{
|
||||
for (auto & replica : replicas)
|
||||
{
|
||||
if (replica.second->connected())
|
||||
return replica.second;
|
||||
if (replica.second->isConnected())
|
||||
return replica.second->get();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
|
||||
PostgreSQLReplicaConnection(const PostgreSQLReplicaConnection & other);
|
||||
|
||||
PostgreSQLConnectionPoolPtr get();
|
||||
WrappedPostgreSQLConnection get();
|
||||
|
||||
|
||||
private:
|
||||
|
@ -88,7 +88,7 @@ Pipe StoragePostgreSQL::read(
|
||||
}
|
||||
|
||||
return Pipe(std::make_shared<SourceFromInputStream>(
|
||||
std::make_shared<PostgreSQLBlockInputStream>(connection_pool, query, sample_block, max_block_size_)));
|
||||
std::make_shared<PostgreSQLBlockInputStream>(connection_pool->get(), query, sample_block, max_block_size_)));
|
||||
}
|
||||
|
||||
|
||||
@ -97,11 +97,10 @@ class PostgreSQLBlockOutputStream : public IBlockOutputStream
|
||||
public:
|
||||
explicit PostgreSQLBlockOutputStream(
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
||||
WrappedPostgreSQLConnection connection_,
|
||||
const std::string & remote_table_name_)
|
||||
: metadata_snapshot(metadata_snapshot_)
|
||||
, connection_pool(connection_pool_)
|
||||
, connection(connection_pool->get())
|
||||
, connection(connection_)
|
||||
, remote_table_name(remote_table_name_)
|
||||
{
|
||||
}
|
||||
@ -167,9 +166,6 @@ public:
|
||||
stream_inserter->complete();
|
||||
work->commit();
|
||||
}
|
||||
|
||||
if (connection->is_open())
|
||||
connection_pool->put(connection);
|
||||
}
|
||||
|
||||
|
||||
@ -280,8 +276,7 @@ public:
|
||||
|
||||
private:
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
PostgreSQLConnectionPoolPtr connection_pool;
|
||||
PostgreSQLConnection::ConnectionPtr connection;
|
||||
WrappedPostgreSQLConnection connection;
|
||||
std::string remote_table_name;
|
||||
|
||||
std::unique_ptr<pqxx::work> work;
|
||||
@ -292,7 +287,7 @@ private:
|
||||
BlockOutputStreamPtr StoragePostgreSQL::write(
|
||||
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */)
|
||||
{
|
||||
return std::make_shared<PostgreSQLBlockOutputStream>(metadata_snapshot, connection_pool, remote_table_name);
|
||||
return std::make_shared<PostgreSQLBlockOutputStream>(metadata_snapshot, connection_pool->get(), remote_table_name);
|
||||
}
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user