mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into nanodbc
This commit is contained in:
commit
5a143a4d12
@ -8,6 +8,7 @@ add_subdirectory (loggers)
|
|||||||
add_subdirectory (pcg-random)
|
add_subdirectory (pcg-random)
|
||||||
add_subdirectory (widechar_width)
|
add_subdirectory (widechar_width)
|
||||||
add_subdirectory (readpassphrase)
|
add_subdirectory (readpassphrase)
|
||||||
|
add_subdirectory (bridge)
|
||||||
|
|
||||||
if (USE_MYSQL)
|
if (USE_MYSQL)
|
||||||
add_subdirectory (mysqlxx)
|
add_subdirectory (mysqlxx)
|
||||||
|
7
base/bridge/CMakeLists.txt
Normal file
7
base/bridge/CMakeLists.txt
Normal file
@ -0,0 +1,7 @@
|
|||||||
|
add_library (bridge
|
||||||
|
IBridge.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
target_include_directories (daemon PUBLIC ..)
|
||||||
|
target_link_libraries (bridge PRIVATE daemon dbms Poco::Data Poco::Data::ODBC)
|
||||||
|
|
238
base/bridge/IBridge.cpp
Normal file
238
base/bridge/IBridge.cpp
Normal file
@ -0,0 +1,238 @@
|
|||||||
|
#include "IBridge.h"
|
||||||
|
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <boost/program_options.hpp>
|
||||||
|
#include <Poco/Net/NetException.h>
|
||||||
|
#include <Poco/Util/HelpFormatter.h>
|
||||||
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
|
#include <Formats/registerFormats.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <Common/SensitiveDataMasker.h>
|
||||||
|
#include <Server/HTTP/HTTPServer.h>
|
||||||
|
|
||||||
|
#if USE_ODBC
|
||||||
|
# include <Poco/Data/ODBC/Connector.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log)
|
||||||
|
{
|
||||||
|
Poco::Net::SocketAddress socket_address;
|
||||||
|
try
|
||||||
|
{
|
||||||
|
socket_address = Poco::Net::SocketAddress(host, port);
|
||||||
|
}
|
||||||
|
catch (const Poco::Net::DNSException & e)
|
||||||
|
{
|
||||||
|
const auto code = e.code();
|
||||||
|
if (code == EAI_FAMILY
|
||||||
|
#if defined(EAI_ADDRFAMILY)
|
||||||
|
|| code == EAI_ADDRFAMILY
|
||||||
|
#endif
|
||||||
|
)
|
||||||
|
{
|
||||||
|
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host>", host, e.code(), e.message());
|
||||||
|
}
|
||||||
|
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
return socket_address;
|
||||||
|
}
|
||||||
|
|
||||||
|
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log)
|
||||||
|
{
|
||||||
|
auto address = makeSocketAddress(host, port, log);
|
||||||
|
#if POCO_VERSION < 0x01080000
|
||||||
|
socket.bind(address, /* reuseAddress = */ true);
|
||||||
|
#else
|
||||||
|
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ false);
|
||||||
|
#endif
|
||||||
|
|
||||||
|
socket.listen(/* backlog = */ 64);
|
||||||
|
|
||||||
|
return address;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IBridge::handleHelp(const std::string &, const std::string &)
|
||||||
|
{
|
||||||
|
Poco::Util::HelpFormatter help_formatter(options());
|
||||||
|
help_formatter.setCommand(commandName());
|
||||||
|
help_formatter.setHeader("HTTP-proxy for odbc requests");
|
||||||
|
help_formatter.setUsage("--http-port <port>");
|
||||||
|
help_formatter.format(std::cerr);
|
||||||
|
|
||||||
|
stopOptionsProcessing();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IBridge::defineOptions(Poco::Util::OptionSet & options)
|
||||||
|
{
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("http-port", "", "port to listen").argument("http-port", true) .binding("http-port"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("listen-host", "", "hostname or address to listen, default 127.0.0.1").argument("listen-host").binding("listen-host"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("http-timeout", "", "http timeout for socket, default 1800").argument("http-timeout").binding("http-timeout"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024").argument("max-server-connections").binding("max-server-connections"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10").argument("keep-alive-timeout").binding("keep-alive-timeout"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("log-level", "", "sets log level, default info") .argument("log-level").binding("logger.level"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("log-path", "", "log path for all logs, default console").argument("log-path").binding("logger.log"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("err-log-path", "", "err log path for all logs, default no").argument("err-log-path").binding("logger.errorlog"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("stdout-path", "", "stdout log path, default console").argument("stdout-path").binding("logger.stdout"));
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("stderr-path", "", "stderr log path, default console").argument("stderr-path").binding("logger.stderr"));
|
||||||
|
|
||||||
|
using Me = std::decay_t<decltype(*this)>;
|
||||||
|
|
||||||
|
options.addOption(
|
||||||
|
Poco::Util::Option("help", "", "produce this help message").binding("help").callback(Poco::Util::OptionCallback<Me>(this, &Me::handleHelp)));
|
||||||
|
|
||||||
|
ServerApplication::defineOptions(options); // NOLINT Don't need complex BaseDaemon's .xml config
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IBridge::initialize(Application & self)
|
||||||
|
{
|
||||||
|
BaseDaemon::closeFDs();
|
||||||
|
is_help = config().has("help");
|
||||||
|
|
||||||
|
if (is_help)
|
||||||
|
return;
|
||||||
|
|
||||||
|
config().setString("logger", bridgeName());
|
||||||
|
|
||||||
|
/// Redirect stdout, stderr to specified files.
|
||||||
|
/// Some libraries and sanitizers write to stderr in case of errors.
|
||||||
|
const auto stdout_path = config().getString("logger.stdout", "");
|
||||||
|
if (!stdout_path.empty())
|
||||||
|
{
|
||||||
|
if (!freopen(stdout_path.c_str(), "a+", stdout))
|
||||||
|
throw Poco::OpenFileException("Cannot attach stdout to " + stdout_path);
|
||||||
|
|
||||||
|
/// Disable buffering for stdout.
|
||||||
|
setbuf(stdout, nullptr);
|
||||||
|
}
|
||||||
|
const auto stderr_path = config().getString("logger.stderr", "");
|
||||||
|
if (!stderr_path.empty())
|
||||||
|
{
|
||||||
|
if (!freopen(stderr_path.c_str(), "a+", stderr))
|
||||||
|
throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path);
|
||||||
|
|
||||||
|
/// Disable buffering for stderr.
|
||||||
|
setbuf(stderr, nullptr);
|
||||||
|
}
|
||||||
|
|
||||||
|
buildLoggers(config(), logger(), self.commandName());
|
||||||
|
|
||||||
|
BaseDaemon::logRevision();
|
||||||
|
|
||||||
|
log = &logger();
|
||||||
|
hostname = config().getString("listen-host", "127.0.0.1");
|
||||||
|
port = config().getUInt("http-port");
|
||||||
|
if (port > 0xFFFF)
|
||||||
|
throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||||
|
|
||||||
|
http_timeout = config().getUInt("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
|
||||||
|
max_server_connections = config().getUInt("max-server-connections", 1024);
|
||||||
|
keep_alive_timeout = config().getUInt("keep-alive-timeout", 10);
|
||||||
|
|
||||||
|
initializeTerminationAndSignalProcessing();
|
||||||
|
|
||||||
|
#if USE_ODBC
|
||||||
|
if (bridgeName() == "ODBCBridge")
|
||||||
|
Poco::Data::ODBC::Connector::registerConnector();
|
||||||
|
#endif
|
||||||
|
|
||||||
|
ServerApplication::initialize(self); // NOLINT
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IBridge::uninitialize()
|
||||||
|
{
|
||||||
|
BaseDaemon::uninitialize();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
int IBridge::main(const std::vector<std::string> & /*args*/)
|
||||||
|
{
|
||||||
|
if (is_help)
|
||||||
|
return Application::EXIT_OK;
|
||||||
|
|
||||||
|
registerFormats();
|
||||||
|
LOG_INFO(log, "Starting up {} on host: {}, port: {}", bridgeName(), hostname, port);
|
||||||
|
|
||||||
|
Poco::Net::ServerSocket socket;
|
||||||
|
auto address = socketBindListen(socket, hostname, port, log);
|
||||||
|
socket.setReceiveTimeout(http_timeout);
|
||||||
|
socket.setSendTimeout(http_timeout);
|
||||||
|
|
||||||
|
Poco::ThreadPool server_pool(3, max_server_connections);
|
||||||
|
|
||||||
|
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
||||||
|
http_params->setTimeout(http_timeout);
|
||||||
|
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
||||||
|
|
||||||
|
auto shared_context = Context::createShared();
|
||||||
|
Context context(Context::createGlobal(shared_context.get()));
|
||||||
|
context.makeGlobalContext();
|
||||||
|
|
||||||
|
if (config().has("query_masking_rules"))
|
||||||
|
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
|
||||||
|
|
||||||
|
auto server = HTTPServer(
|
||||||
|
context,
|
||||||
|
getHandlerFactoryPtr(context),
|
||||||
|
server_pool,
|
||||||
|
socket,
|
||||||
|
http_params);
|
||||||
|
|
||||||
|
SCOPE_EXIT({
|
||||||
|
LOG_DEBUG(log, "Received termination signal.");
|
||||||
|
LOG_DEBUG(log, "Waiting for current connections to close.");
|
||||||
|
|
||||||
|
server.stop();
|
||||||
|
|
||||||
|
for (size_t count : ext::range(1, 6))
|
||||||
|
{
|
||||||
|
if (server.currentConnections() == 0)
|
||||||
|
break;
|
||||||
|
LOG_DEBUG(log, "Waiting for {} connections, try {}", server.currentConnections(), count);
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
server.start();
|
||||||
|
LOG_INFO(log, "Listening http://{}", address.toString());
|
||||||
|
|
||||||
|
waitForTerminationRequest();
|
||||||
|
return Application::EXIT_OK;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
50
base/bridge/IBridge.h
Normal file
50
base/bridge/IBridge.h
Normal file
@ -0,0 +1,50 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||||
|
#include <Poco/Util/ServerApplication.h>
|
||||||
|
#include <Poco/Logger.h>
|
||||||
|
#include <daemon/BaseDaemon.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Class represents base for clickhouse-odbc-bridge and clickhouse-library-bridge servers.
|
||||||
|
/// Listens to incoming HTTP POST and GET requests on specified port and host.
|
||||||
|
/// Has two handlers '/' for all incoming POST requests and /ping for GET request about service status.
|
||||||
|
class IBridge : public BaseDaemon
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
/// Define command line arguments
|
||||||
|
void defineOptions(Poco::Util::OptionSet & options) override;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
using HandlerFactoryPtr = std::shared_ptr<HTTPRequestHandlerFactory>;
|
||||||
|
|
||||||
|
void initialize(Application & self) override;
|
||||||
|
|
||||||
|
void uninitialize() override;
|
||||||
|
|
||||||
|
int main(const std::vector<std::string> & args) override;
|
||||||
|
|
||||||
|
virtual const std::string bridgeName() const = 0;
|
||||||
|
|
||||||
|
virtual HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const = 0;
|
||||||
|
|
||||||
|
size_t keep_alive_timeout;
|
||||||
|
|
||||||
|
private:
|
||||||
|
void handleHelp(const std::string &, const std::string &);
|
||||||
|
|
||||||
|
bool is_help;
|
||||||
|
std::string hostname;
|
||||||
|
size_t port;
|
||||||
|
std::string log_level;
|
||||||
|
size_t max_server_connections;
|
||||||
|
size_t http_timeout;
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
};
|
||||||
|
}
|
@ -2,7 +2,6 @@
|
|||||||
#include <ctime>
|
#include <ctime>
|
||||||
#include <random>
|
#include <random>
|
||||||
#include <thread>
|
#include <thread>
|
||||||
|
|
||||||
#include <mysqlxx/PoolWithFailover.h>
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
|
|
||||||
|
|
||||||
@ -15,9 +14,12 @@ static bool startsWith(const std::string & s, const char * prefix)
|
|||||||
|
|
||||||
using namespace mysqlxx;
|
using namespace mysqlxx;
|
||||||
|
|
||||||
PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & config_,
|
PoolWithFailover::PoolWithFailover(
|
||||||
const std::string & config_name_, const unsigned default_connections_,
|
const Poco::Util::AbstractConfiguration & config_,
|
||||||
const unsigned max_connections_, const size_t max_tries_)
|
const std::string & config_name_,
|
||||||
|
const unsigned default_connections_,
|
||||||
|
const unsigned max_connections_,
|
||||||
|
const size_t max_tries_)
|
||||||
: max_tries(max_tries_)
|
: max_tries(max_tries_)
|
||||||
{
|
{
|
||||||
shareable = config_.getBool(config_name_ + ".share_connection", false);
|
shareable = config_.getBool(config_name_ + ".share_connection", false);
|
||||||
@ -59,16 +61,38 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & con
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
PoolWithFailover::PoolWithFailover(const std::string & config_name_, const unsigned default_connections_,
|
|
||||||
const unsigned max_connections_, const size_t max_tries_)
|
PoolWithFailover::PoolWithFailover(
|
||||||
: PoolWithFailover{
|
const std::string & config_name_,
|
||||||
Poco::Util::Application::instance().config(), config_name_,
|
const unsigned default_connections_,
|
||||||
default_connections_, max_connections_, max_tries_}
|
const unsigned max_connections_,
|
||||||
|
const size_t max_tries_)
|
||||||
|
: PoolWithFailover{Poco::Util::Application::instance().config(),
|
||||||
|
config_name_, default_connections_, max_connections_, max_tries_}
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
PoolWithFailover::PoolWithFailover(
|
||||||
|
const std::string & database,
|
||||||
|
const RemoteDescription & addresses,
|
||||||
|
const std::string & user,
|
||||||
|
const std::string & password,
|
||||||
|
size_t max_tries_)
|
||||||
|
: max_tries(max_tries_)
|
||||||
|
, shareable(false)
|
||||||
|
{
|
||||||
|
/// Replicas have the same priority, but traversed replicas are moved to the end of the queue.
|
||||||
|
for (const auto & [host, port] : addresses)
|
||||||
|
{
|
||||||
|
replicas_by_priority[0].emplace_back(std::make_shared<Pool>(database, host, user, password, port));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
|
PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
|
||||||
: max_tries{other.max_tries}, shareable{other.shareable}
|
: max_tries{other.max_tries}
|
||||||
|
, shareable{other.shareable}
|
||||||
{
|
{
|
||||||
if (shareable)
|
if (shareable)
|
||||||
{
|
{
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
namespace mysqlxx
|
namespace mysqlxx
|
||||||
{
|
{
|
||||||
/** MySQL connection pool with support of failover.
|
/** MySQL connection pool with support of failover.
|
||||||
|
*
|
||||||
|
* For dictionary source:
|
||||||
* Have information about replicas and their priorities.
|
* Have information about replicas and their priorities.
|
||||||
* Tries to connect to replica in an order of priority. When equal priority, choose replica with maximum time without connections.
|
* Tries to connect to replica in an order of priority. When equal priority, choose replica with maximum time without connections.
|
||||||
*
|
*
|
||||||
@ -68,42 +70,58 @@ namespace mysqlxx
|
|||||||
using PoolPtr = std::shared_ptr<Pool>;
|
using PoolPtr = std::shared_ptr<Pool>;
|
||||||
using Replicas = std::vector<PoolPtr>;
|
using Replicas = std::vector<PoolPtr>;
|
||||||
|
|
||||||
/// [priority][index] -> replica.
|
/// [priority][index] -> replica. Highest priority is 0.
|
||||||
using ReplicasByPriority = std::map<int, Replicas>;
|
using ReplicasByPriority = std::map<int, Replicas>;
|
||||||
|
|
||||||
ReplicasByPriority replicas_by_priority;
|
ReplicasByPriority replicas_by_priority;
|
||||||
|
|
||||||
/// Number of connection tries.
|
/// Number of connection tries.
|
||||||
size_t max_tries;
|
size_t max_tries;
|
||||||
/// Mutex for set of replicas.
|
/// Mutex for set of replicas.
|
||||||
std::mutex mutex;
|
std::mutex mutex;
|
||||||
|
|
||||||
/// Can the Pool be shared
|
/// Can the Pool be shared
|
||||||
bool shareable;
|
bool shareable;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
using Entry = Pool::Entry;
|
using Entry = Pool::Entry;
|
||||||
|
using RemoteDescription = std::vector<std::pair<std::string, uint16_t>>;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* config_name Name of parameter in configuration file.
|
* * Mysql dictionary sourse related params:
|
||||||
|
* config_name Name of parameter in configuration file for dictionary source.
|
||||||
|
*
|
||||||
|
* * Mysql storage related parameters:
|
||||||
|
* replicas_description
|
||||||
|
*
|
||||||
|
* * Mutual parameters:
|
||||||
* default_connections Number of connection in pool to each replica at start.
|
* default_connections Number of connection in pool to each replica at start.
|
||||||
* max_connections Maximum number of connections in pool to each replica.
|
* max_connections Maximum number of connections in pool to each replica.
|
||||||
* max_tries_ Max number of connection tries.
|
* max_tries_ Max number of connection tries.
|
||||||
*/
|
*/
|
||||||
PoolWithFailover(const std::string & config_name_,
|
PoolWithFailover(
|
||||||
|
const std::string & config_name_,
|
||||||
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
|
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
|
||||||
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
|
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
|
||||||
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
||||||
|
|
||||||
PoolWithFailover(const Poco::Util::AbstractConfiguration & config_,
|
PoolWithFailover(
|
||||||
|
const Poco::Util::AbstractConfiguration & config_,
|
||||||
const std::string & config_name_,
|
const std::string & config_name_,
|
||||||
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
|
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
|
||||||
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
|
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
|
||||||
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
||||||
|
|
||||||
|
PoolWithFailover(
|
||||||
|
const std::string & database,
|
||||||
|
const RemoteDescription & addresses,
|
||||||
|
const std::string & user,
|
||||||
|
const std::string & password,
|
||||||
|
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
||||||
|
|
||||||
PoolWithFailover(const PoolWithFailover & other);
|
PoolWithFailover(const PoolWithFailover & other);
|
||||||
|
|
||||||
/** Allocates a connection to use. */
|
/** Allocates a connection to use. */
|
||||||
Entry get();
|
Entry get();
|
||||||
};
|
};
|
||||||
|
|
||||||
|
using PoolWithFailoverPtr = std::shared_ptr<PoolWithFailover>;
|
||||||
}
|
}
|
||||||
|
5
contrib/CMakeLists.txt
vendored
5
contrib/CMakeLists.txt
vendored
@ -47,7 +47,10 @@ add_subdirectory (lz4-cmake)
|
|||||||
add_subdirectory (murmurhash)
|
add_subdirectory (murmurhash)
|
||||||
add_subdirectory (replxx-cmake)
|
add_subdirectory (replxx-cmake)
|
||||||
add_subdirectory (unixodbc-cmake)
|
add_subdirectory (unixodbc-cmake)
|
||||||
add_subdirectory (xz)
|
|
||||||
|
if (USE_INTERNAL_XZ_LIBRARY)
|
||||||
|
add_subdirectory (xz)
|
||||||
|
endif()
|
||||||
|
|
||||||
add_subdirectory (poco-cmake)
|
add_subdirectory (poco-cmake)
|
||||||
add_subdirectory (croaring-cmake)
|
add_subdirectory (croaring-cmake)
|
||||||
|
1
debian/clickhouse-common-static.install
vendored
1
debian/clickhouse-common-static.install
vendored
@ -1,5 +1,6 @@
|
|||||||
usr/bin/clickhouse
|
usr/bin/clickhouse
|
||||||
usr/bin/clickhouse-odbc-bridge
|
usr/bin/clickhouse-odbc-bridge
|
||||||
|
usr/bin/clickhouse-library-bridge
|
||||||
usr/bin/clickhouse-extract-from-config
|
usr/bin/clickhouse-extract-from-config
|
||||||
usr/share/bash-completion/completions
|
usr/share/bash-completion/completions
|
||||||
etc/security/limits.d/clickhouse.conf
|
etc/security/limits.d/clickhouse.conf
|
||||||
|
@ -19,7 +19,8 @@ RUN apt-get update \
|
|||||||
tar \
|
tar \
|
||||||
krb5-user \
|
krb5-user \
|
||||||
iproute2 \
|
iproute2 \
|
||||||
lsof
|
lsof \
|
||||||
|
g++
|
||||||
RUN rm -rf \
|
RUN rm -rf \
|
||||||
/var/lib/apt/lists/* \
|
/var/lib/apt/lists/* \
|
||||||
/var/cache/debconf \
|
/var/cache/debconf \
|
||||||
|
@ -31,6 +31,7 @@ RUN apt-get update \
|
|||||||
software-properties-common \
|
software-properties-common \
|
||||||
libkrb5-dev \
|
libkrb5-dev \
|
||||||
krb5-user \
|
krb5-user \
|
||||||
|
g++ \
|
||||||
&& rm -rf \
|
&& rm -rf \
|
||||||
/var/lib/apt/lists/* \
|
/var/lib/apt/lists/* \
|
||||||
/var/cache/debconf \
|
/var/cache/debconf \
|
||||||
|
@ -0,0 +1,23 @@
|
|||||||
|
version: '2.3'
|
||||||
|
services:
|
||||||
|
mysql2:
|
||||||
|
image: mysql:5.7
|
||||||
|
restart: always
|
||||||
|
environment:
|
||||||
|
MYSQL_ROOT_PASSWORD: clickhouse
|
||||||
|
ports:
|
||||||
|
- 3348:3306
|
||||||
|
mysql3:
|
||||||
|
image: mysql:5.7
|
||||||
|
restart: always
|
||||||
|
environment:
|
||||||
|
MYSQL_ROOT_PASSWORD: clickhouse
|
||||||
|
ports:
|
||||||
|
- 3388:3306
|
||||||
|
mysql4:
|
||||||
|
image: mysql:5.7
|
||||||
|
restart: always
|
||||||
|
environment:
|
||||||
|
MYSQL_ROOT_PASSWORD: clickhouse
|
||||||
|
ports:
|
||||||
|
- 3368:3306
|
@ -11,10 +11,3 @@ services:
|
|||||||
default:
|
default:
|
||||||
aliases:
|
aliases:
|
||||||
- postgre-sql.local
|
- postgre-sql.local
|
||||||
postgres2:
|
|
||||||
image: postgres
|
|
||||||
restart: always
|
|
||||||
environment:
|
|
||||||
POSTGRES_PASSWORD: mysecretpassword
|
|
||||||
ports:
|
|
||||||
- 5441:5432
|
|
||||||
|
@ -0,0 +1,23 @@
|
|||||||
|
version: '2.3'
|
||||||
|
services:
|
||||||
|
postgres2:
|
||||||
|
image: postgres
|
||||||
|
restart: always
|
||||||
|
environment:
|
||||||
|
POSTGRES_PASSWORD: mysecretpassword
|
||||||
|
ports:
|
||||||
|
- 5421:5432
|
||||||
|
postgres3:
|
||||||
|
image: postgres
|
||||||
|
restart: always
|
||||||
|
environment:
|
||||||
|
POSTGRES_PASSWORD: mysecretpassword
|
||||||
|
ports:
|
||||||
|
- 5441:5432
|
||||||
|
postgres4:
|
||||||
|
image: postgres
|
||||||
|
restart: always
|
||||||
|
environment:
|
||||||
|
POSTGRES_PASSWORD: mysecretpassword
|
||||||
|
ports:
|
||||||
|
- 5461:5432
|
@ -21,6 +21,7 @@ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse
|
|||||||
export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse
|
export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse
|
||||||
export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config
|
export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config
|
||||||
export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge
|
export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge
|
||||||
|
export CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH=/clickhouse-library-bridge
|
||||||
|
|
||||||
export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest}
|
export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest}
|
||||||
export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest}
|
export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest}
|
||||||
|
@ -3,7 +3,7 @@ toc_priority: 35
|
|||||||
toc_title: AggregatingMergeTree
|
toc_title: AggregatingMergeTree
|
||||||
---
|
---
|
||||||
|
|
||||||
# Aggregatingmergetree {#aggregatingmergetree}
|
# AggregatingMergeTree {#aggregatingmergetree}
|
||||||
|
|
||||||
The engine inherits from [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree), altering the logic for data parts merging. ClickHouse replaces all rows with the same primary key (or more accurately, with the same [sorting key](../../../engines/table-engines/mergetree-family/mergetree.md)) with a single row (within a one data part) that stores a combination of states of aggregate functions.
|
The engine inherits from [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree), altering the logic for data parts merging. ClickHouse replaces all rows with the same primary key (or more accurately, with the same [sorting key](../../../engines/table-engines/mergetree-family/mergetree.md)) with a single row (within a one data part) that stores a combination of states of aggregate functions.
|
||||||
|
|
||||||
|
5
docs/en/interfaces/third-party/gui.md
vendored
5
docs/en/interfaces/third-party/gui.md
vendored
@ -184,4 +184,9 @@ SeekTable is [free](https://www.seektable.com/help/cloud-pricing) for personal/i
|
|||||||
|
|
||||||
[How to configure ClickHouse connection in SeekTable.](https://www.seektable.com/help/clickhouse-pivot-table)
|
[How to configure ClickHouse connection in SeekTable.](https://www.seektable.com/help/clickhouse-pivot-table)
|
||||||
|
|
||||||
|
|
||||||
|
### Chadmin {#chadmin}
|
||||||
|
|
||||||
|
[Chadmin](https://github.com/bun4uk/chadmin) is a simple UI where you can visualize your currently running queries on your ClickHouse cluster and info about them and kill them if you want.
|
||||||
|
|
||||||
[Original article](https://clickhouse.tech/docs/en/interfaces/third-party/gui/) <!--hide-->
|
[Original article](https://clickhouse.tech/docs/en/interfaces/third-party/gui/) <!--hide-->
|
||||||
|
@ -13,6 +13,7 @@ toc_title: Adopters
|
|||||||
| <a href="https://2gis.ru" class="favicon">2gis</a> | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) |
|
| <a href="https://2gis.ru" class="favicon">2gis</a> | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) |
|
||||||
| <a href="https://getadmiral.com/" class="favicon">Admiral</a> | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) |
|
| <a href="https://getadmiral.com/" class="favicon">Admiral</a> | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) |
|
||||||
| <a href="http://www.adscribe.tv/" class="favicon">AdScribe</a> | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) |
|
| <a href="http://www.adscribe.tv/" class="favicon">AdScribe</a> | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) |
|
||||||
|
| <a href="https://ahrefs.com/" class="favicon">Ahrefs</a> | SEO | Analytics | — | — | [Job listing](https://ahrefs.com/jobs/data-scientist-search) |
|
||||||
| <a href="https://cn.aliyun.com/" class="favicon">Alibaba Cloud</a> | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) |
|
| <a href="https://cn.aliyun.com/" class="favicon">Alibaba Cloud</a> | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) |
|
||||||
| <a href="https://alohabrowser.com/" class="favicon">Aloha Browser</a> | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) |
|
| <a href="https://alohabrowser.com/" class="favicon">Aloha Browser</a> | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) |
|
||||||
| <a href="https://altinity.com/" class="favicon">Altinity</a> | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) |
|
| <a href="https://altinity.com/" class="favicon">Altinity</a> | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) |
|
||||||
|
@ -2844,4 +2844,15 @@ Sets the interval in seconds after which periodically refreshed [live view](../.
|
|||||||
|
|
||||||
Default value: `60`.
|
Default value: `60`.
|
||||||
|
|
||||||
|
## check_query_single_value_result {#check_query_single_value_result}
|
||||||
|
|
||||||
|
Defines the level of detail for the [CHECK TABLE](../../sql-reference/statements/check-table.md#checking-mergetree-tables) query result for `MergeTree` family engines .
|
||||||
|
|
||||||
|
Possible values:
|
||||||
|
|
||||||
|
- 0 — the query shows a check status for every individual data part of a table.
|
||||||
|
- 1 — the query shows the general table check status.
|
||||||
|
|
||||||
|
Default value: `0`.
|
||||||
|
|
||||||
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->
|
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->
|
||||||
|
@ -30,9 +30,36 @@ Performed over the tables with another table engines causes an exception.
|
|||||||
|
|
||||||
Engines from the `*Log` family don’t provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner.
|
Engines from the `*Log` family don’t provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner.
|
||||||
|
|
||||||
For `MergeTree` family engines, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server.
|
## Checking the MergeTree Family Tables {#checking-mergetree-tables}
|
||||||
|
|
||||||
**If the data is corrupted**
|
For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server.
|
||||||
|
|
||||||
|
```sql
|
||||||
|
SET check_query_single_value_result = 0;
|
||||||
|
CHECK TABLE test_table;
|
||||||
|
```
|
||||||
|
|
||||||
|
```text
|
||||||
|
┌─part_path─┬─is_passed─┬─message─┐
|
||||||
|
│ all_1_4_1 │ 1 │ │
|
||||||
|
│ all_1_4_2 │ 1 │ │
|
||||||
|
└───────────┴───────────┴─────────┘
|
||||||
|
```
|
||||||
|
|
||||||
|
If `check_query_single_value_result` = 0, the `CHECK TABLE` query shows the general table check status.
|
||||||
|
|
||||||
|
```sql
|
||||||
|
SET check_query_single_value_result = 1;
|
||||||
|
CHECK TABLE test_table;
|
||||||
|
```
|
||||||
|
|
||||||
|
```text
|
||||||
|
┌─result─┐
|
||||||
|
│ 1 │
|
||||||
|
└────────┘
|
||||||
|
```
|
||||||
|
|
||||||
|
## If the Data Is Corrupted {#if-data-is-corrupted}
|
||||||
|
|
||||||
If the table is corrupted, you can copy the non-corrupted data to another table. To do this:
|
If the table is corrupted, you can copy the non-corrupted data to another table. To do this:
|
||||||
|
|
||||||
|
@ -2746,4 +2746,15 @@ SELECT * FROM test2;
|
|||||||
|
|
||||||
Значение по умолчанию: `60`.
|
Значение по умолчанию: `60`.
|
||||||
|
|
||||||
|
## check_query_single_value_result {#check_query_single_value_result}
|
||||||
|
|
||||||
|
Определяет уровень детализации результата для запросов [CHECK TABLE](../../sql-reference/statements/check-table.md#checking-mergetree-tables) для таблиц семейства `MergeTree`.
|
||||||
|
|
||||||
|
Возможные значения:
|
||||||
|
|
||||||
|
- 0 — запрос возвращает статус каждого куска данных таблицы.
|
||||||
|
- 1 — запрос возвращает статус таблицы в целом.
|
||||||
|
|
||||||
|
Значение по умолчанию: `0`.
|
||||||
|
|
||||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->
|
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->
|
||||||
|
@ -29,9 +29,36 @@ CHECK TABLE [db.]name
|
|||||||
|
|
||||||
В движках `*Log` не предусмотрено автоматическое восстановление данных после сбоя. Используйте запрос `CHECK TABLE`, чтобы своевременно выявлять повреждение данных.
|
В движках `*Log` не предусмотрено автоматическое восстановление данных после сбоя. Используйте запрос `CHECK TABLE`, чтобы своевременно выявлять повреждение данных.
|
||||||
|
|
||||||
Для движков из семейства `MergeTree` запрос `CHECK TABLE` показывает статус проверки для каждого отдельного куска данных таблицы на локальном сервере.
|
## Проверка таблиц семейства MergeTree {#checking-mergetree-tables}
|
||||||
|
|
||||||
**Что делать, если данные повреждены**
|
Для таблиц семейства `MergeTree` если [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, запрос `CHECK TABLE` возвращает статус каждого куска данных таблицы на локальном сервере.
|
||||||
|
|
||||||
|
```sql
|
||||||
|
SET check_query_single_value_result = 0;
|
||||||
|
CHECK TABLE test_table;
|
||||||
|
```
|
||||||
|
|
||||||
|
```text
|
||||||
|
┌─part_path─┬─is_passed─┬─message─┐
|
||||||
|
│ all_1_4_1 │ 1 │ │
|
||||||
|
│ all_1_4_2 │ 1 │ │
|
||||||
|
└───────────┴───────────┴─────────┘
|
||||||
|
```
|
||||||
|
|
||||||
|
Если `check_query_single_value_result` = 0, запрос `CHECK TABLE` возвращает статус таблицы в целом.
|
||||||
|
|
||||||
|
```sql
|
||||||
|
SET check_query_single_value_result = 1;
|
||||||
|
CHECK TABLE test_table;
|
||||||
|
```
|
||||||
|
|
||||||
|
```text
|
||||||
|
┌─result─┐
|
||||||
|
│ 1 │
|
||||||
|
└────────┘
|
||||||
|
```
|
||||||
|
|
||||||
|
## Что делать, если данные повреждены {#if-data-is-corrupted}
|
||||||
|
|
||||||
В этом случае можно скопировать оставшиеся неповрежденные данные в другую таблицу. Для этого:
|
В этом случае можно скопировать оставшиеся неповрежденные данные в другую таблицу. Для этого:
|
||||||
|
|
||||||
|
@ -36,6 +36,9 @@ option (ENABLE_CLICKHOUSE_OBFUSCATOR "Table data obfuscator (convert real data t
|
|||||||
option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "HTTP-server working like a proxy to ODBC driver"
|
option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "HTTP-server working like a proxy to ODBC driver"
|
||||||
${ENABLE_CLICKHOUSE_ALL})
|
${ENABLE_CLICKHOUSE_ALL})
|
||||||
|
|
||||||
|
option (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE "HTTP-server working like a proxy to Library dictionary source"
|
||||||
|
${ENABLE_CLICKHOUSE_ALL})
|
||||||
|
|
||||||
# https://presentations.clickhouse.tech/matemarketing_2020/
|
# https://presentations.clickhouse.tech/matemarketing_2020/
|
||||||
option (ENABLE_CLICKHOUSE_GIT_IMPORT "A tool to analyze Git repositories"
|
option (ENABLE_CLICKHOUSE_GIT_IMPORT "A tool to analyze Git repositories"
|
||||||
${ENABLE_CLICKHOUSE_ALL})
|
${ENABLE_CLICKHOUSE_ALL})
|
||||||
@ -109,6 +112,12 @@ else()
|
|||||||
message(STATUS "ODBC bridge mode: OFF")
|
message(STATUS "ODBC bridge mode: OFF")
|
||||||
endif()
|
endif()
|
||||||
|
|
||||||
|
if (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE)
|
||||||
|
message(STATUS "Library bridge mode: ON")
|
||||||
|
else()
|
||||||
|
message(STATUS "Library bridge mode: OFF")
|
||||||
|
endif()
|
||||||
|
|
||||||
if (ENABLE_CLICKHOUSE_INSTALL)
|
if (ENABLE_CLICKHOUSE_INSTALL)
|
||||||
message(STATUS "ClickHouse install: ON")
|
message(STATUS "ClickHouse install: ON")
|
||||||
else()
|
else()
|
||||||
@ -194,6 +203,10 @@ if (ENABLE_CLICKHOUSE_ODBC_BRIDGE)
|
|||||||
add_subdirectory (odbc-bridge)
|
add_subdirectory (odbc-bridge)
|
||||||
endif ()
|
endif ()
|
||||||
|
|
||||||
|
if (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE)
|
||||||
|
add_subdirectory (library-bridge)
|
||||||
|
endif ()
|
||||||
|
|
||||||
if (CLICKHOUSE_ONE_SHARED)
|
if (CLICKHOUSE_ONE_SHARED)
|
||||||
add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_GIT_IMPORT_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES})
|
add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_GIT_IMPORT_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES})
|
||||||
target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_GIT_IMPORT_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK})
|
target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_GIT_IMPORT_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK})
|
||||||
@ -209,6 +222,10 @@ if (CLICKHOUSE_SPLIT_BINARY)
|
|||||||
list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge)
|
list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge)
|
||||||
endif ()
|
endif ()
|
||||||
|
|
||||||
|
if (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE)
|
||||||
|
list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-library-bridge)
|
||||||
|
endif ()
|
||||||
|
|
||||||
set_target_properties(${CLICKHOUSE_ALL_TARGETS} PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..)
|
set_target_properties(${CLICKHOUSE_ALL_TARGETS} PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..)
|
||||||
|
|
||||||
add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_ALL_TARGETS})
|
add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_ALL_TARGETS})
|
||||||
|
@ -15,3 +15,4 @@
|
|||||||
#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT
|
#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT
|
||||||
#cmakedefine01 ENABLE_CLICKHOUSE_INSTALL
|
#cmakedefine01 ENABLE_CLICKHOUSE_INSTALL
|
||||||
#cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE
|
#cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE
|
||||||
|
#cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE
|
||||||
|
26
programs/library-bridge/CMakeLists.txt
Normal file
26
programs/library-bridge/CMakeLists.txt
Normal file
@ -0,0 +1,26 @@
|
|||||||
|
set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES
|
||||||
|
library-bridge.cpp
|
||||||
|
library-log.cpp
|
||||||
|
LibraryBridge.cpp
|
||||||
|
Handlers.cpp
|
||||||
|
HandlerFactory.cpp
|
||||||
|
SharedLibraryHandler.cpp
|
||||||
|
SharedLibraryHandlerFactory.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
if (OS_LINUX)
|
||||||
|
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic")
|
||||||
|
endif ()
|
||||||
|
|
||||||
|
add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES})
|
||||||
|
|
||||||
|
target_link_libraries(clickhouse-library-bridge PRIVATE
|
||||||
|
daemon
|
||||||
|
dbms
|
||||||
|
clickhouse_parsers
|
||||||
|
bridge
|
||||||
|
)
|
||||||
|
|
||||||
|
set_target_properties(clickhouse-library-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..)
|
||||||
|
|
||||||
|
install(TARGETS clickhouse-library-bridge RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
|
23
programs/library-bridge/HandlerFactory.cpp
Normal file
23
programs/library-bridge/HandlerFactory.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "HandlerFactory.h"
|
||||||
|
|
||||||
|
#include <Poco/Net/HTTPServerRequest.h>
|
||||||
|
#include <Server/HTTP/HTMLForm.h>
|
||||||
|
#include "Handlers.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
std::unique_ptr<HTTPRequestHandler> LibraryBridgeHandlerFactory::createRequestHandler(const HTTPServerRequest & request)
|
||||||
|
{
|
||||||
|
Poco::URI uri{request.getURI()};
|
||||||
|
LOG_DEBUG(log, "Request URI: {}", uri.toString());
|
||||||
|
|
||||||
|
if (uri == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
|
||||||
|
return std::make_unique<PingHandler>(keep_alive_timeout);
|
||||||
|
|
||||||
|
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
|
||||||
|
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context);
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
}
|
38
programs/library-bridge/HandlerFactory.h
Normal file
38
programs/library-bridge/HandlerFactory.h
Normal file
@ -0,0 +1,38 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class SharedLibraryHandler;
|
||||||
|
using SharedLibraryHandlerPtr = std::shared_ptr<SharedLibraryHandler>;
|
||||||
|
|
||||||
|
/// Factory for '/ping', '/' handlers.
|
||||||
|
class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
LibraryBridgeHandlerFactory(
|
||||||
|
const std::string & name_,
|
||||||
|
size_t keep_alive_timeout_,
|
||||||
|
Context & context_)
|
||||||
|
: log(&Poco::Logger::get(name_))
|
||||||
|
, name(name_)
|
||||||
|
, keep_alive_timeout(keep_alive_timeout_)
|
||||||
|
, context(context_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
Poco::Logger * log;
|
||||||
|
std::string name;
|
||||||
|
size_t keep_alive_timeout;
|
||||||
|
Context & context;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
288
programs/library-bridge/Handlers.cpp
Normal file
288
programs/library-bridge/Handlers.cpp
Normal file
@ -0,0 +1,288 @@
|
|||||||
|
#include "Handlers.h"
|
||||||
|
#include "SharedLibraryHandlerFactory.h"
|
||||||
|
|
||||||
|
#include <DataStreams/copyData.h>
|
||||||
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <Poco/Net/HTTPServerRequest.h>
|
||||||
|
#include <Poco/Net/HTTPServerResponse.h>
|
||||||
|
#include <Poco/Net/HTMLForm.h>
|
||||||
|
#include <Poco/ThreadPool.h>
|
||||||
|
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||||
|
#include <Server/HTTP/HTMLForm.h>
|
||||||
|
#include <IO/ReadBufferFromString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
std::shared_ptr<Block> parseColumns(std::string && column_string)
|
||||||
|
{
|
||||||
|
auto sample_block = std::make_shared<Block>();
|
||||||
|
auto names_and_types = NamesAndTypesList::parse(column_string);
|
||||||
|
|
||||||
|
for (const NameAndTypePair & column_data : names_and_types)
|
||||||
|
sample_block->insert({column_data.type, column_data.name});
|
||||||
|
|
||||||
|
return sample_block;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<uint64_t> parseIdsFromBinary(const std::string & ids_string)
|
||||||
|
{
|
||||||
|
ReadBufferFromString buf(ids_string);
|
||||||
|
std::vector<uint64_t> ids;
|
||||||
|
readVectorBinary(ids, buf);
|
||||||
|
return ids;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<std::string> parseNamesFromBinary(const std::string & names_string)
|
||||||
|
{
|
||||||
|
ReadBufferFromString buf(names_string);
|
||||||
|
std::vector<std::string> names;
|
||||||
|
readVectorBinary(names, buf);
|
||||||
|
return names;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||||
|
{
|
||||||
|
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||||
|
HTMLForm params(request);
|
||||||
|
|
||||||
|
if (!params.has("method"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'method' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!params.has("dictionary_id"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'dictionary_id in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string method = params.get("method");
|
||||||
|
std::string dictionary_id = params.get("dictionary_id");
|
||||||
|
LOG_TRACE(log, "Library method: '{}', dictionary id: {}", method, dictionary_id);
|
||||||
|
|
||||||
|
WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout);
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
if (method == "libNew")
|
||||||
|
{
|
||||||
|
auto & read_buf = request.getStream();
|
||||||
|
params.read(read_buf);
|
||||||
|
|
||||||
|
if (!params.has("library_path"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'library_path' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!params.has("library_settings"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'library_settings' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string library_path = params.get("library_path");
|
||||||
|
const auto & settings_string = params.get("library_settings");
|
||||||
|
std::vector<std::string> library_settings = parseNamesFromBinary(settings_string);
|
||||||
|
|
||||||
|
/// Needed for library dictionary
|
||||||
|
if (!params.has("attributes_names"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'attributes_names' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto & attributes_string = params.get("attributes_names");
|
||||||
|
std::vector<std::string> attributes_names = parseNamesFromBinary(attributes_string);
|
||||||
|
|
||||||
|
/// Needed to parse block from binary string format
|
||||||
|
if (!params.has("sample_block"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'sample_block' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
std::string sample_block_string = params.get("sample_block");
|
||||||
|
|
||||||
|
std::shared_ptr<Block> sample_block;
|
||||||
|
try
|
||||||
|
{
|
||||||
|
sample_block = parseColumns(std::move(sample_block_string));
|
||||||
|
}
|
||||||
|
catch (const Exception & ex)
|
||||||
|
{
|
||||||
|
processError(response, "Invalid 'sample_block' parameter in request body '" + ex.message() + "'");
|
||||||
|
LOG_WARNING(log, ex.getStackTraceString());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!params.has("null_values"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'null_values' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
ReadBufferFromString read_block_buf(params.get("null_values"));
|
||||||
|
auto format = FormatFactory::instance().getInput(FORMAT, read_block_buf, *sample_block, context, DEFAULT_BLOCK_SIZE);
|
||||||
|
auto reader = std::make_shared<InputStreamFromInputFormat>(format);
|
||||||
|
auto sample_block_with_nulls = reader->read();
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Dictionary sample block with null values: {}", sample_block_with_nulls.dumpStructure());
|
||||||
|
|
||||||
|
SharedLibraryHandlerFactory::instance().create(dictionary_id, library_path, library_settings, sample_block_with_nulls, attributes_names);
|
||||||
|
writeStringBinary("1", out);
|
||||||
|
}
|
||||||
|
else if (method == "libClone")
|
||||||
|
{
|
||||||
|
if (!params.has("from_dictionary_id"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'from_dictionary_id' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string from_dictionary_id = params.get("from_dictionary_id");
|
||||||
|
LOG_TRACE(log, "Calling libClone from {} to {}", from_dictionary_id, dictionary_id);
|
||||||
|
SharedLibraryHandlerFactory::instance().clone(from_dictionary_id, dictionary_id);
|
||||||
|
writeStringBinary("1", out);
|
||||||
|
}
|
||||||
|
else if (method == "libDelete")
|
||||||
|
{
|
||||||
|
SharedLibraryHandlerFactory::instance().remove(dictionary_id);
|
||||||
|
writeStringBinary("1", out);
|
||||||
|
}
|
||||||
|
else if (method == "isModified")
|
||||||
|
{
|
||||||
|
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||||
|
bool res = library_handler->isModified();
|
||||||
|
writeStringBinary(std::to_string(res), out);
|
||||||
|
}
|
||||||
|
else if (method == "supportsSelectiveLoad")
|
||||||
|
{
|
||||||
|
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||||
|
bool res = library_handler->supportsSelectiveLoad();
|
||||||
|
writeStringBinary(std::to_string(res), out);
|
||||||
|
}
|
||||||
|
else if (method == "loadAll")
|
||||||
|
{
|
||||||
|
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||||
|
const auto & sample_block = library_handler->getSampleBlock();
|
||||||
|
auto input = library_handler->loadAll();
|
||||||
|
|
||||||
|
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, context);
|
||||||
|
copyData(*input, *output);
|
||||||
|
}
|
||||||
|
else if (method == "loadIds")
|
||||||
|
{
|
||||||
|
params.read(request.getStream());
|
||||||
|
|
||||||
|
if (!params.has("ids"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'ids' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<uint64_t> ids = parseIdsFromBinary(params.get("ids"));
|
||||||
|
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||||
|
const auto & sample_block = library_handler->getSampleBlock();
|
||||||
|
auto input = library_handler->loadIds(ids);
|
||||||
|
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, context);
|
||||||
|
copyData(*input, *output);
|
||||||
|
}
|
||||||
|
else if (method == "loadKeys")
|
||||||
|
{
|
||||||
|
if (!params.has("requested_block_sample"))
|
||||||
|
{
|
||||||
|
processError(response, "No 'requested_block_sample' in request URL");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string requested_block_string = params.get("requested_block_sample");
|
||||||
|
|
||||||
|
std::shared_ptr<Block> requested_sample_block;
|
||||||
|
try
|
||||||
|
{
|
||||||
|
requested_sample_block = parseColumns(std::move(requested_block_string));
|
||||||
|
}
|
||||||
|
catch (const Exception & ex)
|
||||||
|
{
|
||||||
|
processError(response, "Invalid 'requested_block' parameter in request body '" + ex.message() + "'");
|
||||||
|
LOG_WARNING(log, ex.getStackTraceString());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto & read_buf = request.getStream();
|
||||||
|
auto format = FormatFactory::instance().getInput(FORMAT, read_buf, *requested_sample_block, context, DEFAULT_BLOCK_SIZE);
|
||||||
|
auto reader = std::make_shared<InputStreamFromInputFormat>(format);
|
||||||
|
auto block = reader->read();
|
||||||
|
|
||||||
|
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||||
|
const auto & sample_block = library_handler->getSampleBlock();
|
||||||
|
auto input = library_handler->loadKeys(block.getColumns());
|
||||||
|
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, context);
|
||||||
|
copyData(*input, *output);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
auto message = getCurrentExceptionMessage(true);
|
||||||
|
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR, message); // can't call process_error, because of too soon response sending
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
writeStringBinary(message, out);
|
||||||
|
out.finalize();
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
tryLogCurrentException(log);
|
||||||
|
}
|
||||||
|
|
||||||
|
tryLogCurrentException(log);
|
||||||
|
}
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
out.finalize();
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
tryLogCurrentException(log);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void LibraryRequestHandler::processError(HTTPServerResponse & response, const std::string & message)
|
||||||
|
{
|
||||||
|
response.setStatusAndReason(HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
|
||||||
|
|
||||||
|
if (!response.sent())
|
||||||
|
*response.send() << message << std::endl;
|
||||||
|
|
||||||
|
LOG_WARNING(log, message);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void PingHandler::handleRequest(HTTPServerRequest & /* request */, HTTPServerResponse & response)
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
setResponseDefaultHeaders(response, keep_alive_timeout);
|
||||||
|
const char * data = "Ok.\n";
|
||||||
|
response.sendBuffer(data, strlen(data));
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
tryLogCurrentException("PingHandler");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
60
programs/library-bridge/Handlers.h
Normal file
60
programs/library-bridge/Handlers.h
Normal file
@ -0,0 +1,60 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Server/HTTP/HTTPRequestHandler.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include "SharedLibraryHandler.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
/// Handler for requests to Library Dictionary Source, returns response in RowBinary format.
|
||||||
|
/// When a library dictionary source is created, it sends libNew request to library bridge (which is started on first
|
||||||
|
/// request to it, if it was not yet started). On this request a new sharedLibrayHandler is added to a
|
||||||
|
/// sharedLibraryHandlerFactory by a dictionary uuid. With libNew request come: library_path, library_settings,
|
||||||
|
/// names of dictionary attributes, sample block to parse block of null values, block of null values. Everything is
|
||||||
|
/// passed in binary format and is urlencoded. When dictionary is cloned, a new handler is created.
|
||||||
|
/// Each handler is unique to dictionary.
|
||||||
|
class LibraryRequestHandler : public HTTPRequestHandler
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
|
||||||
|
LibraryRequestHandler(
|
||||||
|
size_t keep_alive_timeout_,
|
||||||
|
Context & context_)
|
||||||
|
: log(&Poco::Logger::get("LibraryRequestHandler"))
|
||||||
|
, keep_alive_timeout(keep_alive_timeout_)
|
||||||
|
, context(context_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
static constexpr inline auto FORMAT = "RowBinary";
|
||||||
|
|
||||||
|
void processError(HTTPServerResponse & response, const std::string & message);
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
size_t keep_alive_timeout;
|
||||||
|
Context & context;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
class PingHandler : public HTTPRequestHandler
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit PingHandler(size_t keep_alive_timeout_)
|
||||||
|
: keep_alive_timeout(keep_alive_timeout_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
const size_t keep_alive_timeout;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
17
programs/library-bridge/LibraryBridge.cpp
Normal file
17
programs/library-bridge/LibraryBridge.cpp
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
#include "LibraryBridge.h"
|
||||||
|
|
||||||
|
#pragma GCC diagnostic ignored "-Wmissing-declarations"
|
||||||
|
int mainEntryClickHouseLibraryBridge(int argc, char ** argv)
|
||||||
|
{
|
||||||
|
DB::LibraryBridge app;
|
||||||
|
try
|
||||||
|
{
|
||||||
|
return app.run(argc, argv);
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
|
||||||
|
auto code = DB::getCurrentExceptionCode();
|
||||||
|
return code ? code : 1;
|
||||||
|
}
|
||||||
|
}
|
26
programs/library-bridge/LibraryBridge.h
Normal file
26
programs/library-bridge/LibraryBridge.h
Normal file
@ -0,0 +1,26 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <bridge/IBridge.h>
|
||||||
|
#include "HandlerFactory.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class LibraryBridge : public IBridge
|
||||||
|
{
|
||||||
|
|
||||||
|
protected:
|
||||||
|
const std::string bridgeName() const override
|
||||||
|
{
|
||||||
|
return "LibraryBridge";
|
||||||
|
}
|
||||||
|
|
||||||
|
HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const override
|
||||||
|
{
|
||||||
|
return std::make_shared<LibraryBridgeHandlerFactory>("LibraryRequestHandlerFactory-factory", keep_alive_timeout, context);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
43
programs/library-bridge/LibraryUtils.h
Normal file
43
programs/library-bridge/LibraryUtils.h
Normal file
@ -0,0 +1,43 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
|
#include <Dictionaries/LibraryDictionarySourceExternal.h>
|
||||||
|
#include <Core/Block.h>
|
||||||
|
#include <ext/bit_cast.h>
|
||||||
|
#include <ext/range.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class CStringsHolder
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
using Container = std::vector<std::string>;
|
||||||
|
|
||||||
|
explicit CStringsHolder(const Container & strings_pass)
|
||||||
|
{
|
||||||
|
strings_holder = strings_pass;
|
||||||
|
strings.size = strings_holder.size();
|
||||||
|
|
||||||
|
ptr_holder = std::make_unique<ClickHouseLibrary::CString[]>(strings.size);
|
||||||
|
strings.data = ptr_holder.get();
|
||||||
|
|
||||||
|
size_t i = 0;
|
||||||
|
for (auto & str : strings_holder)
|
||||||
|
{
|
||||||
|
strings.data[i] = str.c_str();
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
ClickHouseLibrary::CStrings strings; // will pass pointer to lib
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::unique_ptr<ClickHouseLibrary::CString[]> ptr_holder = nullptr;
|
||||||
|
Container strings_holder;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
}
|
219
programs/library-bridge/SharedLibraryHandler.cpp
Normal file
219
programs/library-bridge/SharedLibraryHandler.cpp
Normal file
@ -0,0 +1,219 @@
|
|||||||
|
#include "SharedLibraryHandler.h"
|
||||||
|
|
||||||
|
#include <ext/scope_guard.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <common/find_symbols.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int EXTERNAL_LIBRARY_ERROR;
|
||||||
|
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
SharedLibraryHandler::SharedLibraryHandler(
|
||||||
|
const std::string & library_path_,
|
||||||
|
const std::vector<std::string> & library_settings,
|
||||||
|
const Block & sample_block_,
|
||||||
|
const std::vector<std::string> & attributes_names_)
|
||||||
|
: library_path(library_path_)
|
||||||
|
, sample_block(sample_block_)
|
||||||
|
, attributes_names(attributes_names_)
|
||||||
|
{
|
||||||
|
library = std::make_shared<SharedLibrary>(library_path, RTLD_LAZY);
|
||||||
|
settings_holder = std::make_shared<CStringsHolder>(CStringsHolder(library_settings));
|
||||||
|
|
||||||
|
auto lib_new = library->tryGet<ClickHouseLibrary::LibraryNewFunc>(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME);
|
||||||
|
|
||||||
|
if (lib_new)
|
||||||
|
lib_data = lib_new(&settings_holder->strings, ClickHouseLibrary::log);
|
||||||
|
else
|
||||||
|
throw Exception("Method libNew failed", ErrorCodes::EXTERNAL_LIBRARY_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
SharedLibraryHandler::SharedLibraryHandler(const SharedLibraryHandler & other)
|
||||||
|
: library_path{other.library_path}
|
||||||
|
, sample_block{other.sample_block}
|
||||||
|
, attributes_names{other.attributes_names}
|
||||||
|
, library{other.library}
|
||||||
|
, settings_holder{other.settings_holder}
|
||||||
|
{
|
||||||
|
|
||||||
|
auto lib_clone = library->tryGet<ClickHouseLibrary::LibraryCloneFunc>(ClickHouseLibrary::LIBRARY_CLONE_FUNC_NAME);
|
||||||
|
|
||||||
|
if (lib_clone)
|
||||||
|
{
|
||||||
|
lib_data = lib_clone(other.lib_data);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto lib_new = library->tryGet<ClickHouseLibrary::LibraryNewFunc>(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME);
|
||||||
|
|
||||||
|
if (lib_new)
|
||||||
|
lib_data = lib_new(&settings_holder->strings, ClickHouseLibrary::log);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
SharedLibraryHandler::~SharedLibraryHandler()
|
||||||
|
{
|
||||||
|
auto lib_delete = library->tryGet<ClickHouseLibrary::LibraryDeleteFunc>(ClickHouseLibrary::LIBRARY_DELETE_FUNC_NAME);
|
||||||
|
|
||||||
|
if (lib_delete)
|
||||||
|
lib_delete(lib_data);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool SharedLibraryHandler::isModified()
|
||||||
|
{
|
||||||
|
auto func_is_modified = library->tryGet<ClickHouseLibrary::LibraryIsModifiedFunc>(ClickHouseLibrary::LIBRARY_IS_MODIFIED_FUNC_NAME);
|
||||||
|
|
||||||
|
if (func_is_modified)
|
||||||
|
return func_is_modified(lib_data, &settings_holder->strings);
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool SharedLibraryHandler::supportsSelectiveLoad()
|
||||||
|
{
|
||||||
|
auto func_supports_selective_load = library->tryGet<ClickHouseLibrary::LibrarySupportsSelectiveLoadFunc>(ClickHouseLibrary::LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME);
|
||||||
|
|
||||||
|
if (func_supports_selective_load)
|
||||||
|
return func_supports_selective_load(lib_data, &settings_holder->strings);
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr SharedLibraryHandler::loadAll()
|
||||||
|
{
|
||||||
|
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(attributes_names.size());
|
||||||
|
ClickHouseLibrary::CStrings columns{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()), attributes_names.size()};
|
||||||
|
for (size_t i = 0; i < attributes_names.size(); ++i)
|
||||||
|
columns.data[i] = attributes_names[i].c_str();
|
||||||
|
|
||||||
|
auto load_all_func = library->get<ClickHouseLibrary::LibraryLoadAllFunc>(ClickHouseLibrary::LIBRARY_LOAD_ALL_FUNC_NAME);
|
||||||
|
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
|
||||||
|
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME);
|
||||||
|
|
||||||
|
ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data);
|
||||||
|
SCOPE_EXIT(data_delete_func(lib_data, data_ptr));
|
||||||
|
|
||||||
|
ClickHouseLibrary::RawClickHouseLibraryTable data = load_all_func(data_ptr, &settings_holder->strings, &columns);
|
||||||
|
auto block = dataToBlock(data);
|
||||||
|
|
||||||
|
return std::make_shared<OneBlockInputStream>(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr SharedLibraryHandler::loadIds(const std::vector<uint64_t> & ids)
|
||||||
|
{
|
||||||
|
const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast<decltype(ClickHouseLibrary::VectorUInt64::data)>(ids.data()), ids.size()};
|
||||||
|
|
||||||
|
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(attributes_names.size());
|
||||||
|
ClickHouseLibrary::CStrings columns_pass{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()), attributes_names.size()};
|
||||||
|
|
||||||
|
auto load_ids_func = library->get<ClickHouseLibrary::LibraryLoadIdsFunc>(ClickHouseLibrary::LIBRARY_LOAD_IDS_FUNC_NAME);
|
||||||
|
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
|
||||||
|
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME);
|
||||||
|
|
||||||
|
ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data);
|
||||||
|
SCOPE_EXIT(data_delete_func(lib_data, data_ptr));
|
||||||
|
|
||||||
|
ClickHouseLibrary::RawClickHouseLibraryTable data = load_ids_func(data_ptr, &settings_holder->strings, &columns_pass, &ids_data);
|
||||||
|
auto block = dataToBlock(data);
|
||||||
|
|
||||||
|
return std::make_shared<OneBlockInputStream>(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr SharedLibraryHandler::loadKeys(const Columns & key_columns)
|
||||||
|
{
|
||||||
|
auto holder = std::make_unique<ClickHouseLibrary::Row[]>(key_columns.size());
|
||||||
|
std::vector<std::unique_ptr<ClickHouseLibrary::Field[]>> column_data_holders;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < key_columns.size(); ++i)
|
||||||
|
{
|
||||||
|
auto cell_holder = std::make_unique<ClickHouseLibrary::Field[]>(key_columns[i]->size());
|
||||||
|
|
||||||
|
for (size_t j = 0; j < key_columns[i]->size(); ++j)
|
||||||
|
{
|
||||||
|
auto data_ref = key_columns[i]->getDataAt(j);
|
||||||
|
|
||||||
|
cell_holder[j] = ClickHouseLibrary::Field{
|
||||||
|
.data = static_cast<const void *>(data_ref.data),
|
||||||
|
.size = data_ref.size};
|
||||||
|
}
|
||||||
|
|
||||||
|
holder[i] = ClickHouseLibrary::Row{
|
||||||
|
.data = static_cast<ClickHouseLibrary::Field *>(cell_holder.get()),
|
||||||
|
.size = key_columns[i]->size()};
|
||||||
|
|
||||||
|
column_data_holders.push_back(std::move(cell_holder));
|
||||||
|
}
|
||||||
|
|
||||||
|
ClickHouseLibrary::Table request_cols{
|
||||||
|
.data = static_cast<ClickHouseLibrary::Row *>(holder.get()),
|
||||||
|
.size = key_columns.size()};
|
||||||
|
|
||||||
|
auto load_keys_func = library->get<ClickHouseLibrary::LibraryLoadKeysFunc>(ClickHouseLibrary::LIBRARY_LOAD_KEYS_FUNC_NAME);
|
||||||
|
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
|
||||||
|
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME);
|
||||||
|
|
||||||
|
ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data);
|
||||||
|
SCOPE_EXIT(data_delete_func(lib_data, data_ptr));
|
||||||
|
|
||||||
|
ClickHouseLibrary::RawClickHouseLibraryTable data = load_keys_func(data_ptr, &settings_holder->strings, &request_cols);
|
||||||
|
auto block = dataToBlock(data);
|
||||||
|
|
||||||
|
return std::make_shared<OneBlockInputStream>(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Block SharedLibraryHandler::dataToBlock(const ClickHouseLibrary::RawClickHouseLibraryTable data)
|
||||||
|
{
|
||||||
|
if (!data)
|
||||||
|
throw Exception("LibraryDictionarySource: No data returned", ErrorCodes::EXTERNAL_LIBRARY_ERROR);
|
||||||
|
|
||||||
|
const auto * columns_received = static_cast<const ClickHouseLibrary::Table *>(data);
|
||||||
|
if (columns_received->error_code)
|
||||||
|
throw Exception(
|
||||||
|
"LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " " + (columns_received->error_string ? columns_received->error_string : ""),
|
||||||
|
ErrorCodes::EXTERNAL_LIBRARY_ERROR);
|
||||||
|
|
||||||
|
MutableColumns columns = sample_block.cloneEmptyColumns();
|
||||||
|
|
||||||
|
for (size_t col_n = 0; col_n < columns_received->size; ++col_n)
|
||||||
|
{
|
||||||
|
if (columns.size() != columns_received->data[col_n].size)
|
||||||
|
throw Exception(
|
||||||
|
"LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size) + ", must be " + std::to_string(columns.size()),
|
||||||
|
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n)
|
||||||
|
{
|
||||||
|
const auto & field = columns_received->data[col_n].data[row_n];
|
||||||
|
if (!field.data)
|
||||||
|
{
|
||||||
|
/// sample_block contains null_value (from config) inside corresponding column
|
||||||
|
const auto & col = sample_block.getByPosition(row_n);
|
||||||
|
columns[row_n]->insertFrom(*(col.column), 0);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
const auto & size = field.size;
|
||||||
|
columns[row_n]->insertData(static_cast<const char *>(field.data), size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return sample_block.cloneWithColumns(std::move(columns));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
54
programs/library-bridge/SharedLibraryHandler.h
Normal file
54
programs/library-bridge/SharedLibraryHandler.h
Normal file
@ -0,0 +1,54 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/SharedLibrary.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <DataStreams/OneBlockInputStream.h>
|
||||||
|
#include "LibraryUtils.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// A class that manages all operations with library dictionary.
|
||||||
|
/// Every library dictionary source has its own object of this class, accessed by UUID.
|
||||||
|
class SharedLibraryHandler
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
SharedLibraryHandler(
|
||||||
|
const std::string & library_path_,
|
||||||
|
const std::vector<std::string> & library_settings,
|
||||||
|
const Block & sample_block_,
|
||||||
|
const std::vector<std::string> & attributes_names_);
|
||||||
|
|
||||||
|
SharedLibraryHandler(const SharedLibraryHandler & other);
|
||||||
|
|
||||||
|
~SharedLibraryHandler();
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadAll();
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadIds(const std::vector<uint64_t> & ids);
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadKeys(const Columns & key_columns);
|
||||||
|
|
||||||
|
bool isModified();
|
||||||
|
|
||||||
|
bool supportsSelectiveLoad();
|
||||||
|
|
||||||
|
const Block & getSampleBlock() { return sample_block; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
Block dataToBlock(const ClickHouseLibrary::RawClickHouseLibraryTable data);
|
||||||
|
|
||||||
|
std::string library_path;
|
||||||
|
const Block sample_block;
|
||||||
|
std::vector<std::string> attributes_names;
|
||||||
|
|
||||||
|
SharedLibraryPtr library;
|
||||||
|
std::shared_ptr<CStringsHolder> settings_holder;
|
||||||
|
void * lib_data;
|
||||||
|
};
|
||||||
|
|
||||||
|
using SharedLibraryHandlerPtr = std::shared_ptr<SharedLibraryHandler>;
|
||||||
|
|
||||||
|
}
|
67
programs/library-bridge/SharedLibraryHandlerFactory.cpp
Normal file
67
programs/library-bridge/SharedLibraryHandlerFactory.cpp
Normal file
@ -0,0 +1,67 @@
|
|||||||
|
#include "SharedLibraryHandlerFactory.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
|
SharedLibraryHandlerPtr SharedLibraryHandlerFactory::get(const std::string & dictionary_id)
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
auto library_handler = library_handlers.find(dictionary_id);
|
||||||
|
|
||||||
|
if (library_handler != library_handlers.end())
|
||||||
|
return library_handler->second;
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void SharedLibraryHandlerFactory::create(
|
||||||
|
const std::string & dictionary_id,
|
||||||
|
const std::string & library_path,
|
||||||
|
const std::vector<std::string> & library_settings,
|
||||||
|
const Block & sample_block,
|
||||||
|
const std::vector<std::string> & attributes_names)
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
library_handlers[dictionary_id] = std::make_shared<SharedLibraryHandler>(library_path, library_settings, sample_block, attributes_names);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void SharedLibraryHandlerFactory::clone(const std::string & from_dictionary_id, const std::string & to_dictionary_id)
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
auto from_library_handler = library_handlers.find(from_dictionary_id);
|
||||||
|
|
||||||
|
/// This is not supposed to happen as libClone is called from copy constructor of LibraryDictionarySource
|
||||||
|
/// object, and shared library handler of from_dictionary is removed only in its destructor.
|
||||||
|
/// And if for from_dictionary there was no shared library handler, it would have received and exception in
|
||||||
|
/// its constructor, so no libClone would be made from it.
|
||||||
|
if (from_library_handler == library_handlers.end())
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No shared library handler found");
|
||||||
|
|
||||||
|
/// libClone method will be called in copy constructor
|
||||||
|
library_handlers[to_dictionary_id] = std::make_shared<SharedLibraryHandler>(*from_library_handler->second);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void SharedLibraryHandlerFactory::remove(const std::string & dictionary_id)
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
/// libDelete is called in destructor.
|
||||||
|
library_handlers.erase(dictionary_id);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
SharedLibraryHandlerFactory & SharedLibraryHandlerFactory::instance()
|
||||||
|
{
|
||||||
|
static SharedLibraryHandlerFactory ret;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
37
programs/library-bridge/SharedLibraryHandlerFactory.h
Normal file
37
programs/library-bridge/SharedLibraryHandlerFactory.h
Normal file
@ -0,0 +1,37 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include "SharedLibraryHandler.h"
|
||||||
|
#include <unordered_map>
|
||||||
|
#include <mutex>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Each library dictionary source has unique UUID. When clone() method is called, a new UUID is generated.
|
||||||
|
/// There is a unique mapping from diciotnary UUID to sharedLibraryHandler.
|
||||||
|
class SharedLibraryHandlerFactory final : private boost::noncopyable
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static SharedLibraryHandlerFactory & instance();
|
||||||
|
|
||||||
|
SharedLibraryHandlerPtr get(const std::string & dictionary_id);
|
||||||
|
|
||||||
|
void create(
|
||||||
|
const std::string & dictionary_id,
|
||||||
|
const std::string & library_path,
|
||||||
|
const std::vector<std::string> & library_settings,
|
||||||
|
const Block & sample_block,
|
||||||
|
const std::vector<std::string> & attributes_names);
|
||||||
|
|
||||||
|
void clone(const std::string & from_dictionary_id, const std::string & to_dictionary_id);
|
||||||
|
|
||||||
|
void remove(const std::string & dictionary_id);
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// map: dict_id -> sharedLibraryHandler
|
||||||
|
std::unordered_map<std::string, SharedLibraryHandlerPtr> library_handlers;
|
||||||
|
std::mutex mutex;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
3
programs/library-bridge/library-bridge.cpp
Normal file
3
programs/library-bridge/library-bridge.cpp
Normal file
@ -0,0 +1,3 @@
|
|||||||
|
int mainEntryClickHouseLibraryBridge(int argc, char ** argv);
|
||||||
|
int main(int argc_, char ** argv_) { return mainEntryClickHouseLibraryBridge(argc_, argv_); }
|
||||||
|
|
66
programs/library-bridge/library-log.cpp
Normal file
66
programs/library-bridge/library-log.cpp
Normal file
@ -0,0 +1,66 @@
|
|||||||
|
#include <Dictionaries/LibraryDictionarySourceExternal.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
const char DICT_LOGGER_NAME[] = "LibraryDictionarySourceExternal";
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace ClickHouseLibrary
|
||||||
|
{
|
||||||
|
|
||||||
|
std::string_view LIBRARY_CREATE_NEW_FUNC_NAME = "ClickHouseDictionary_v3_libNew";
|
||||||
|
std::string_view LIBRARY_CLONE_FUNC_NAME = "ClickHouseDictionary_v3_libClone";
|
||||||
|
std::string_view LIBRARY_DELETE_FUNC_NAME = "ClickHouseDictionary_v3_libDelete";
|
||||||
|
|
||||||
|
std::string_view LIBRARY_DATA_NEW_FUNC_NAME = "ClickHouseDictionary_v3_dataNew";
|
||||||
|
std::string_view LIBRARY_DATA_DELETE_FUNC_NAME = "ClickHouseDictionary_v3_dataDelete";
|
||||||
|
|
||||||
|
std::string_view LIBRARY_LOAD_ALL_FUNC_NAME = "ClickHouseDictionary_v3_loadAll";
|
||||||
|
std::string_view LIBRARY_LOAD_IDS_FUNC_NAME = "ClickHouseDictionary_v3_loadIds";
|
||||||
|
std::string_view LIBRARY_LOAD_KEYS_FUNC_NAME = "ClickHouseDictionary_v3_loadKeys";
|
||||||
|
|
||||||
|
std::string_view LIBRARY_IS_MODIFIED_FUNC_NAME = "ClickHouseDictionary_v3_isModified";
|
||||||
|
std::string_view LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME = "ClickHouseDictionary_v3_supportsSelectiveLoad";
|
||||||
|
|
||||||
|
void log(LogLevel level, CString msg)
|
||||||
|
{
|
||||||
|
auto & logger = Poco::Logger::get(DICT_LOGGER_NAME);
|
||||||
|
switch (level)
|
||||||
|
{
|
||||||
|
case LogLevel::TRACE:
|
||||||
|
if (logger.trace())
|
||||||
|
logger.trace(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::DEBUG:
|
||||||
|
if (logger.debug())
|
||||||
|
logger.debug(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::INFORMATION:
|
||||||
|
if (logger.information())
|
||||||
|
logger.information(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::NOTICE:
|
||||||
|
if (logger.notice())
|
||||||
|
logger.notice(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::WARNING:
|
||||||
|
if (logger.warning())
|
||||||
|
logger.warning(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::ERROR:
|
||||||
|
if (logger.error())
|
||||||
|
logger.error(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::CRITICAL:
|
||||||
|
if (logger.critical())
|
||||||
|
logger.critical(msg);
|
||||||
|
break;
|
||||||
|
case LogLevel::FATAL:
|
||||||
|
if (logger.fatal())
|
||||||
|
logger.fatal(msg);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -24,6 +24,7 @@ add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES})
|
|||||||
target_link_libraries(clickhouse-odbc-bridge PRIVATE
|
target_link_libraries(clickhouse-odbc-bridge PRIVATE
|
||||||
daemon
|
daemon
|
||||||
dbms
|
dbms
|
||||||
|
bridge
|
||||||
clickhouse_parsers
|
clickhouse_parsers
|
||||||
Poco::Data
|
Poco::Data
|
||||||
nanodbc
|
nanodbc
|
||||||
|
@ -8,7 +8,7 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
std::unique_ptr<HTTPRequestHandler> HandlerFactory::createRequestHandler(const HTTPServerRequest & request)
|
std::unique_ptr<HTTPRequestHandler> ODBCBridgeHandlerFactory::createRequestHandler(const HTTPServerRequest & request)
|
||||||
{
|
{
|
||||||
Poco::URI uri{request.getURI()};
|
Poco::URI uri{request.getURI()};
|
||||||
LOG_TRACE(log, "Request URI: {}", uri.toString());
|
LOG_TRACE(log, "Request URI: {}", uri.toString());
|
||||||
|
@ -14,10 +14,10 @@ namespace DB
|
|||||||
/** Factory for '/ping', '/', '/columns_info', '/identifier_quote', '/schema_allowed' handlers.
|
/** Factory for '/ping', '/', '/columns_info', '/identifier_quote', '/schema_allowed' handlers.
|
||||||
* Also stores Session pools for ODBC connections
|
* Also stores Session pools for ODBC connections
|
||||||
*/
|
*/
|
||||||
class HandlerFactory : public HTTPRequestHandlerFactory
|
class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, Context & context_)
|
ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, Context & context_)
|
||||||
: log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_)
|
: log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
@ -30,4 +30,5 @@ private:
|
|||||||
size_t keep_alive_timeout;
|
size_t keep_alive_timeout;
|
||||||
Context & context;
|
Context & context;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,232 +1,4 @@
|
|||||||
#include "ODBCBridge.h"
|
#include "ODBCBridge.h"
|
||||||
#include "HandlerFactory.h"
|
|
||||||
|
|
||||||
#include <string>
|
|
||||||
#include <errno.h>
|
|
||||||
#include <IO/ReadHelpers.h>
|
|
||||||
#include <boost/program_options.hpp>
|
|
||||||
#include <Poco/Net/NetException.h>
|
|
||||||
#include <Poco/String.h>
|
|
||||||
#include <Poco/Util/HelpFormatter.h>
|
|
||||||
#include <Common/Exception.h>
|
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Formats/registerFormats.h>
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
#include <ext/scope_guard.h>
|
|
||||||
#include <ext/range.h>
|
|
||||||
#include <Common/SensitiveDataMasker.h>
|
|
||||||
#include <Server/HTTP/HTTPServer.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
|
||||||
}
|
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log)
|
|
||||||
{
|
|
||||||
Poco::Net::SocketAddress socket_address;
|
|
||||||
try
|
|
||||||
{
|
|
||||||
socket_address = Poco::Net::SocketAddress(host, port);
|
|
||||||
}
|
|
||||||
catch (const Poco::Net::DNSException & e)
|
|
||||||
{
|
|
||||||
const auto code = e.code();
|
|
||||||
if (code == EAI_FAMILY
|
|
||||||
#if defined(EAI_ADDRFAMILY)
|
|
||||||
|| code == EAI_ADDRFAMILY
|
|
||||||
#endif
|
|
||||||
)
|
|
||||||
{
|
|
||||||
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host>", host, e.code(), e.message());
|
|
||||||
}
|
|
||||||
|
|
||||||
throw;
|
|
||||||
}
|
|
||||||
return socket_address;
|
|
||||||
}
|
|
||||||
|
|
||||||
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log)
|
|
||||||
{
|
|
||||||
auto address = makeSocketAddress(host, port, log);
|
|
||||||
#if POCO_VERSION < 0x01080000
|
|
||||||
socket.bind(address, /* reuseAddress = */ true);
|
|
||||||
#else
|
|
||||||
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ false);
|
|
||||||
#endif
|
|
||||||
|
|
||||||
socket.listen(/* backlog = */ 64);
|
|
||||||
|
|
||||||
return address;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void ODBCBridge::handleHelp(const std::string &, const std::string &)
|
|
||||||
{
|
|
||||||
Poco::Util::HelpFormatter help_formatter(options());
|
|
||||||
help_formatter.setCommand(commandName());
|
|
||||||
help_formatter.setHeader("HTTP-proxy for odbc requests");
|
|
||||||
help_formatter.setUsage("--http-port <port>");
|
|
||||||
help_formatter.format(std::cerr);
|
|
||||||
|
|
||||||
stopOptionsProcessing();
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
void ODBCBridge::defineOptions(Poco::Util::OptionSet & options)
|
|
||||||
{
|
|
||||||
options.addOption(Poco::Util::Option("http-port", "", "port to listen").argument("http-port", true).binding("http-port"));
|
|
||||||
options.addOption(
|
|
||||||
Poco::Util::Option("listen-host", "", "hostname or address to listen, default 127.0.0.1").argument("listen-host").binding("listen-host"));
|
|
||||||
options.addOption(
|
|
||||||
Poco::Util::Option("http-timeout", "", "http timeout for socket, default 1800").argument("http-timeout").binding("http-timeout"));
|
|
||||||
|
|
||||||
options.addOption(Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024")
|
|
||||||
.argument("max-server-connections")
|
|
||||||
.binding("max-server-connections"));
|
|
||||||
options.addOption(Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10")
|
|
||||||
.argument("keep-alive-timeout")
|
|
||||||
.binding("keep-alive-timeout"));
|
|
||||||
|
|
||||||
options.addOption(Poco::Util::Option("log-level", "", "sets log level, default info").argument("log-level").binding("logger.level"));
|
|
||||||
|
|
||||||
options.addOption(
|
|
||||||
Poco::Util::Option("log-path", "", "log path for all logs, default console").argument("log-path").binding("logger.log"));
|
|
||||||
|
|
||||||
options.addOption(Poco::Util::Option("err-log-path", "", "err log path for all logs, default no")
|
|
||||||
.argument("err-log-path")
|
|
||||||
.binding("logger.errorlog"));
|
|
||||||
|
|
||||||
options.addOption(Poco::Util::Option("stdout-path", "", "stdout log path, default console")
|
|
||||||
.argument("stdout-path")
|
|
||||||
.binding("logger.stdout"));
|
|
||||||
|
|
||||||
options.addOption(Poco::Util::Option("stderr-path", "", "stderr log path, default console")
|
|
||||||
.argument("stderr-path")
|
|
||||||
.binding("logger.stderr"));
|
|
||||||
|
|
||||||
using Me = std::decay_t<decltype(*this)>;
|
|
||||||
options.addOption(Poco::Util::Option("help", "", "produce this help message")
|
|
||||||
.binding("help")
|
|
||||||
.callback(Poco::Util::OptionCallback<Me>(this, &Me::handleHelp)));
|
|
||||||
|
|
||||||
ServerApplication::defineOptions(options); // NOLINT Don't need complex BaseDaemon's .xml config
|
|
||||||
}
|
|
||||||
|
|
||||||
void ODBCBridge::initialize(Application & self)
|
|
||||||
{
|
|
||||||
BaseDaemon::closeFDs();
|
|
||||||
is_help = config().has("help");
|
|
||||||
|
|
||||||
if (is_help)
|
|
||||||
return;
|
|
||||||
|
|
||||||
config().setString("logger", "ODBCBridge");
|
|
||||||
|
|
||||||
/// Redirect stdout, stderr to specified files.
|
|
||||||
/// Some libraries and sanitizers write to stderr in case of errors.
|
|
||||||
const auto stdout_path = config().getString("logger.stdout", "");
|
|
||||||
if (!stdout_path.empty())
|
|
||||||
{
|
|
||||||
if (!freopen(stdout_path.c_str(), "a+", stdout))
|
|
||||||
throw Poco::OpenFileException("Cannot attach stdout to " + stdout_path);
|
|
||||||
|
|
||||||
/// Disable buffering for stdout.
|
|
||||||
setbuf(stdout, nullptr);
|
|
||||||
}
|
|
||||||
const auto stderr_path = config().getString("logger.stderr", "");
|
|
||||||
if (!stderr_path.empty())
|
|
||||||
{
|
|
||||||
if (!freopen(stderr_path.c_str(), "a+", stderr))
|
|
||||||
throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path);
|
|
||||||
|
|
||||||
/// Disable buffering for stderr.
|
|
||||||
setbuf(stderr, nullptr);
|
|
||||||
}
|
|
||||||
|
|
||||||
buildLoggers(config(), logger(), self.commandName());
|
|
||||||
|
|
||||||
BaseDaemon::logRevision();
|
|
||||||
|
|
||||||
log = &logger();
|
|
||||||
hostname = config().getString("listen-host", "127.0.0.1");
|
|
||||||
port = config().getUInt("http-port");
|
|
||||||
if (port > 0xFFFF)
|
|
||||||
throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
|
||||||
|
|
||||||
http_timeout = config().getUInt("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
|
|
||||||
max_server_connections = config().getUInt("max-server-connections", 1024);
|
|
||||||
keep_alive_timeout = config().getUInt("keep-alive-timeout", 10);
|
|
||||||
|
|
||||||
initializeTerminationAndSignalProcessing();
|
|
||||||
|
|
||||||
ServerApplication::initialize(self); // NOLINT
|
|
||||||
}
|
|
||||||
|
|
||||||
void ODBCBridge::uninitialize()
|
|
||||||
{
|
|
||||||
BaseDaemon::uninitialize();
|
|
||||||
}
|
|
||||||
|
|
||||||
int ODBCBridge::main(const std::vector<std::string> & /*args*/)
|
|
||||||
{
|
|
||||||
if (is_help)
|
|
||||||
return Application::EXIT_OK;
|
|
||||||
|
|
||||||
registerFormats();
|
|
||||||
|
|
||||||
LOG_INFO(log, "Starting up");
|
|
||||||
Poco::Net::ServerSocket socket;
|
|
||||||
auto address = socketBindListen(socket, hostname, port, log);
|
|
||||||
socket.setReceiveTimeout(http_timeout);
|
|
||||||
socket.setSendTimeout(http_timeout);
|
|
||||||
Poco::ThreadPool server_pool(3, max_server_connections);
|
|
||||||
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
|
||||||
http_params->setTimeout(http_timeout);
|
|
||||||
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
|
||||||
|
|
||||||
auto shared_context = Context::createShared();
|
|
||||||
Context context(Context::createGlobal(shared_context.get()));
|
|
||||||
context.makeGlobalContext();
|
|
||||||
|
|
||||||
if (config().has("query_masking_rules"))
|
|
||||||
{
|
|
||||||
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
|
|
||||||
}
|
|
||||||
|
|
||||||
auto server = HTTPServer(
|
|
||||||
context,
|
|
||||||
std::make_shared<HandlerFactory>("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context),
|
|
||||||
server_pool,
|
|
||||||
socket,
|
|
||||||
http_params);
|
|
||||||
server.start();
|
|
||||||
|
|
||||||
LOG_INFO(log, "Listening http://{}", address.toString());
|
|
||||||
|
|
||||||
SCOPE_EXIT({
|
|
||||||
LOG_DEBUG(log, "Received termination signal.");
|
|
||||||
LOG_DEBUG(log, "Waiting for current connections to close.");
|
|
||||||
server.stop();
|
|
||||||
for (size_t count : ext::range(1, 6))
|
|
||||||
{
|
|
||||||
if (server.currentConnections() == 0)
|
|
||||||
break;
|
|
||||||
LOG_DEBUG(log, "Waiting for {} connections, try {}", server.currentConnections(), count);
|
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
waitForTerminationRequest();
|
|
||||||
return Application::EXIT_OK;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
#pragma GCC diagnostic ignored "-Wmissing-declarations"
|
#pragma GCC diagnostic ignored "-Wmissing-declarations"
|
||||||
int mainEntryClickHouseODBCBridge(int argc, char ** argv)
|
int mainEntryClickHouseODBCBridge(int argc, char ** argv)
|
||||||
|
@ -2,39 +2,26 @@
|
|||||||
|
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Poco/Logger.h>
|
#include <Poco/Logger.h>
|
||||||
#include <daemon/BaseDaemon.h>
|
#include <bridge/IBridge.h>
|
||||||
|
#include "HandlerFactory.h"
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
/** Class represents clickhouse-odbc-bridge server, which listen
|
|
||||||
* incoming HTTP POST and GET requests on specified port and host.
|
class ODBCBridge : public IBridge
|
||||||
* Has two handlers '/' for all incoming POST requests to ODBC driver
|
|
||||||
* and /ping for GET request about service status
|
|
||||||
*/
|
|
||||||
class ODBCBridge : public BaseDaemon
|
|
||||||
{
|
{
|
||||||
public:
|
|
||||||
void defineOptions(Poco::Util::OptionSet & options) override;
|
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
void initialize(Application & self) override;
|
const std::string bridgeName() const override
|
||||||
|
{
|
||||||
|
return "ODBCBridge";
|
||||||
|
}
|
||||||
|
|
||||||
void uninitialize() override;
|
HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const override
|
||||||
|
{
|
||||||
int main(const std::vector<std::string> & args) override;
|
return std::make_shared<ODBCBridgeHandlerFactory>("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context);
|
||||||
|
}
|
||||||
private:
|
|
||||||
void handleHelp(const std::string &, const std::string &);
|
|
||||||
|
|
||||||
bool is_help;
|
|
||||||
std::string hostname;
|
|
||||||
size_t port;
|
|
||||||
size_t http_timeout;
|
|
||||||
std::string log_level;
|
|
||||||
size_t max_server_connections;
|
|
||||||
size_t keep_alive_timeout;
|
|
||||||
|
|
||||||
Poco::Logger * log;
|
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
132
src/Bridge/IBridgeHelper.cpp
Normal file
132
src/Bridge/IBridgeHelper.cpp
Normal file
@ -0,0 +1,132 @@
|
|||||||
|
#include "IBridgeHelper.h"
|
||||||
|
|
||||||
|
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
|
#include <Poco/Path.h>
|
||||||
|
#include <Poco/URI.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Poco::URI IBridgeHelper::getMainURI() const
|
||||||
|
{
|
||||||
|
auto uri = createBaseURI();
|
||||||
|
uri.setPath(MAIN_HANDLER);
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Poco::URI IBridgeHelper::getPingURI() const
|
||||||
|
{
|
||||||
|
auto uri = createBaseURI();
|
||||||
|
uri.setPath(PING_HANDLER);
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool IBridgeHelper::checkBridgeIsRunning() const
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
ReadWriteBufferFromHTTP buf(
|
||||||
|
getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, ConnectionTimeouts::getHTTPTimeouts(getContext()));
|
||||||
|
return checkString(PING_OK_ANSWER, buf);
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IBridgeHelper::startBridgeSync() const
|
||||||
|
{
|
||||||
|
if (!checkBridgeIsRunning())
|
||||||
|
{
|
||||||
|
LOG_TRACE(getLog(), "{} is not running, will try to start it", serviceAlias());
|
||||||
|
startBridge(startBridgeCommand());
|
||||||
|
bool started = false;
|
||||||
|
|
||||||
|
uint64_t milliseconds_to_wait = 10; /// Exponential backoff
|
||||||
|
uint64_t counter = 0;
|
||||||
|
|
||||||
|
while (milliseconds_to_wait < 10000)
|
||||||
|
{
|
||||||
|
++counter;
|
||||||
|
LOG_TRACE(getLog(), "Checking {} is running, try {}", serviceAlias(), counter);
|
||||||
|
|
||||||
|
if (checkBridgeIsRunning())
|
||||||
|
{
|
||||||
|
started = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(milliseconds_to_wait));
|
||||||
|
milliseconds_to_wait *= 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!started)
|
||||||
|
throw Exception("BridgeHelper: " + serviceAlias() + " is not responding",
|
||||||
|
ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
std::unique_ptr<ShellCommand> IBridgeHelper::startBridgeCommand() const
|
||||||
|
{
|
||||||
|
if (startBridgeManually())
|
||||||
|
throw Exception(serviceAlias() + " is not running. Please, start it manually", ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING);
|
||||||
|
|
||||||
|
const auto & config = getConfig();
|
||||||
|
/// Path to executable folder
|
||||||
|
Poco::Path path{config.getString("application.dir", "/usr/bin")};
|
||||||
|
|
||||||
|
std::vector<std::string> cmd_args;
|
||||||
|
path.setFileName(serviceFileName());
|
||||||
|
|
||||||
|
cmd_args.push_back("--http-port");
|
||||||
|
cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", getDefaultPort())));
|
||||||
|
cmd_args.push_back("--listen-host");
|
||||||
|
cmd_args.push_back(config.getString(configPrefix() + ".listen_host", DEFAULT_HOST));
|
||||||
|
cmd_args.push_back("--http-timeout");
|
||||||
|
cmd_args.push_back(std::to_string(getHTTPTimeout().totalMicroseconds()));
|
||||||
|
if (config.has("logger." + configPrefix() + "_log"))
|
||||||
|
{
|
||||||
|
cmd_args.push_back("--log-path");
|
||||||
|
cmd_args.push_back(config.getString("logger." + configPrefix() + "_log"));
|
||||||
|
}
|
||||||
|
if (config.has("logger." + configPrefix() + "_errlog"))
|
||||||
|
{
|
||||||
|
cmd_args.push_back("--err-log-path");
|
||||||
|
cmd_args.push_back(config.getString("logger." + configPrefix() + "_errlog"));
|
||||||
|
}
|
||||||
|
if (config.has("logger." + configPrefix() + "_stdout"))
|
||||||
|
{
|
||||||
|
cmd_args.push_back("--stdout-path");
|
||||||
|
cmd_args.push_back(config.getString("logger." + configPrefix() + "_stdout"));
|
||||||
|
}
|
||||||
|
if (config.has("logger." + configPrefix() + "_stderr"))
|
||||||
|
{
|
||||||
|
cmd_args.push_back("--stderr-path");
|
||||||
|
cmd_args.push_back(config.getString("logger." + configPrefix() + "_stderr"));
|
||||||
|
}
|
||||||
|
if (config.has("logger." + configPrefix() + "_level"))
|
||||||
|
{
|
||||||
|
cmd_args.push_back("--log-level");
|
||||||
|
cmd_args.push_back(config.getString("logger." + configPrefix() + "_level"));
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG_TRACE(getLog(), "Starting {}", serviceAlias());
|
||||||
|
|
||||||
|
return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
70
src/Bridge/IBridgeHelper.h
Normal file
70
src/Bridge/IBridgeHelper.h
Normal file
@ -0,0 +1,70 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Poco/Logger.h>
|
||||||
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
|
#include <Common/ShellCommand.h>
|
||||||
|
#include <IO/ConnectionTimeoutsContext.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Common base class for XDBC and Library bridge helpers.
|
||||||
|
/// Contains helper methods to check/start bridge sync.
|
||||||
|
class IBridgeHelper
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
static constexpr inline auto DEFAULT_HOST = "127.0.0.1";
|
||||||
|
static constexpr inline auto PING_HANDLER = "/ping";
|
||||||
|
static constexpr inline auto MAIN_HANDLER = "/";
|
||||||
|
static constexpr inline auto DEFAULT_FORMAT = "RowBinary";
|
||||||
|
static constexpr inline auto PING_OK_ANSWER = "Ok.";
|
||||||
|
|
||||||
|
static const inline std::string PING_METHOD = Poco::Net::HTTPRequest::HTTP_GET;
|
||||||
|
static const inline std::string MAIN_METHOD = Poco::Net::HTTPRequest::HTTP_POST;
|
||||||
|
|
||||||
|
virtual ~IBridgeHelper() = default;
|
||||||
|
|
||||||
|
void startBridgeSync() const;
|
||||||
|
|
||||||
|
Poco::URI getMainURI() const;
|
||||||
|
|
||||||
|
Poco::URI getPingURI() const;
|
||||||
|
|
||||||
|
|
||||||
|
protected:
|
||||||
|
/// clickhouse-odbc-bridge, clickhouse-library-bridge
|
||||||
|
virtual const String serviceAlias() const = 0;
|
||||||
|
|
||||||
|
virtual const String serviceFileName() const = 0;
|
||||||
|
|
||||||
|
virtual size_t getDefaultPort() const = 0;
|
||||||
|
|
||||||
|
virtual bool startBridgeManually() const = 0;
|
||||||
|
|
||||||
|
virtual void startBridge(std::unique_ptr<ShellCommand> cmd) const = 0;
|
||||||
|
|
||||||
|
virtual const String configPrefix() const = 0;
|
||||||
|
|
||||||
|
virtual const Context & getContext() const = 0;
|
||||||
|
|
||||||
|
virtual const Poco::Util::AbstractConfiguration & getConfig() const = 0;
|
||||||
|
|
||||||
|
virtual Poco::Logger * getLog() const = 0;
|
||||||
|
|
||||||
|
virtual const Poco::Timespan & getHTTPTimeout() const = 0;
|
||||||
|
|
||||||
|
virtual Poco::URI createBaseURI() const = 0;
|
||||||
|
|
||||||
|
|
||||||
|
private:
|
||||||
|
bool checkBridgeIsRunning() const;
|
||||||
|
|
||||||
|
std::unique_ptr<ShellCommand> startBridgeCommand() const;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
183
src/Bridge/LibraryBridgeHelper.cpp
Normal file
183
src/Bridge/LibraryBridgeHelper.cpp
Normal file
@ -0,0 +1,183 @@
|
|||||||
|
#include "LibraryBridgeHelper.h"
|
||||||
|
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <DataStreams/OneBlockInputStream.h>
|
||||||
|
#include <DataStreams/OwningBlockInputStream.h>
|
||||||
|
#include <Dictionaries/DictionarySourceHelpers.h>
|
||||||
|
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||||
|
#include <IO/WriteBufferFromOStream.h>
|
||||||
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Poco/Path.h>
|
||||||
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <Common/ShellCommand.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <ext/range.h>
|
||||||
|
#include <Core/Field.h>
|
||||||
|
#include <Common/escapeForFileName.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
LibraryBridgeHelper::LibraryBridgeHelper(
|
||||||
|
const Context & context_,
|
||||||
|
const Block & sample_block_,
|
||||||
|
const Field & dictionary_id_)
|
||||||
|
: log(&Poco::Logger::get("LibraryBridgeHelper"))
|
||||||
|
, context(context_)
|
||||||
|
, sample_block(sample_block_)
|
||||||
|
, config(context.getConfigRef())
|
||||||
|
, http_timeout(context.getSettingsRef().http_receive_timeout.value.totalSeconds())
|
||||||
|
, dictionary_id(dictionary_id_)
|
||||||
|
{
|
||||||
|
bridge_port = config.getUInt("library_bridge.port", DEFAULT_PORT);
|
||||||
|
bridge_host = config.getString("library_bridge.host", DEFAULT_HOST);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Poco::URI LibraryBridgeHelper::createRequestURI(const String & method) const
|
||||||
|
{
|
||||||
|
auto uri = getMainURI();
|
||||||
|
uri.addQueryParameter("dictionary_id", toString(dictionary_id));
|
||||||
|
uri.addQueryParameter("method", method);
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Poco::URI LibraryBridgeHelper::createBaseURI() const
|
||||||
|
{
|
||||||
|
Poco::URI uri;
|
||||||
|
uri.setHost(bridge_host);
|
||||||
|
uri.setPort(bridge_port);
|
||||||
|
uri.setScheme("http");
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void LibraryBridgeHelper::startBridge(std::unique_ptr<ShellCommand> cmd) const
|
||||||
|
{
|
||||||
|
context.addBridgeCommand(std::move(cmd));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryBridgeHelper::initLibrary(const std::string & library_path, const std::string library_settings, const std::string attributes_names)
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(LIB_NEW_METHOD);
|
||||||
|
|
||||||
|
/// Sample block must contain null values
|
||||||
|
WriteBufferFromOwnString out;
|
||||||
|
auto output_stream = context.getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out, sample_block);
|
||||||
|
formatBlock(output_stream, sample_block);
|
||||||
|
auto block_string = out.str();
|
||||||
|
|
||||||
|
auto out_stream_callback = [library_path, library_settings, attributes_names, block_string, this](std::ostream & os)
|
||||||
|
{
|
||||||
|
os << "library_path=" << escapeForFileName(library_path) << "&";
|
||||||
|
os << "library_settings=" << escapeForFileName(library_settings) << "&";
|
||||||
|
os << "attributes_names=" << escapeForFileName(attributes_names) << "&";
|
||||||
|
os << "sample_block=" << escapeForFileName(sample_block.getNamesAndTypesList().toString()) << "&";
|
||||||
|
os << "null_values=" << escapeForFileName(block_string);
|
||||||
|
};
|
||||||
|
return executeRequest(uri, out_stream_callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryBridgeHelper::cloneLibrary(const Field & other_dictionary_id)
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(LIB_CLONE_METHOD);
|
||||||
|
uri.addQueryParameter("from_dictionary_id", toString(other_dictionary_id));
|
||||||
|
return executeRequest(uri);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryBridgeHelper::removeLibrary()
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(LIB_DELETE_METHOD);
|
||||||
|
return executeRequest(uri);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryBridgeHelper::isModified()
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(IS_MODIFIED_METHOD);
|
||||||
|
return executeRequest(uri);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryBridgeHelper::supportsSelectiveLoad()
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(SUPPORTS_SELECTIVE_LOAD_METHOD);
|
||||||
|
return executeRequest(uri);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr LibraryBridgeHelper::loadAll()
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(LOAD_ALL_METHOD);
|
||||||
|
return loadBase(uri);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::string ids_string)
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(LOAD_IDS_METHOD);
|
||||||
|
return loadBase(uri, [ids_string](std::ostream & os) { os << "ids=" << ids_string; });
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr LibraryBridgeHelper::loadKeys(const Block & requested_block)
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
auto uri = createRequestURI(LOAD_KEYS_METHOD);
|
||||||
|
/// Sample block to parse block from callback
|
||||||
|
uri.addQueryParameter("requested_block_sample", requested_block.getNamesAndTypesList().toString());
|
||||||
|
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [requested_block, this](std::ostream & os)
|
||||||
|
{
|
||||||
|
WriteBufferFromOStream out_buffer(os);
|
||||||
|
auto output_stream = context.getOutputStream(
|
||||||
|
LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block);
|
||||||
|
formatBlock(output_stream, requested_block);
|
||||||
|
};
|
||||||
|
return loadBase(uri, out_stream_callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback)
|
||||||
|
{
|
||||||
|
ReadWriteBufferFromHTTP buf(
|
||||||
|
uri,
|
||||||
|
Poco::Net::HTTPRequest::HTTP_POST,
|
||||||
|
std::move(out_stream_callback),
|
||||||
|
ConnectionTimeouts::getHTTPTimeouts(context));
|
||||||
|
|
||||||
|
bool res;
|
||||||
|
readBoolText(res, buf);
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback)
|
||||||
|
{
|
||||||
|
auto read_buf_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
|
||||||
|
uri,
|
||||||
|
Poco::Net::HTTPRequest::HTTP_POST,
|
||||||
|
std::move(out_stream_callback),
|
||||||
|
ConnectionTimeouts::getHTTPTimeouts(context),
|
||||||
|
0,
|
||||||
|
Poco::Net::HTTPBasicCredentials{},
|
||||||
|
DBMS_DEFAULT_BUFFER_SIZE,
|
||||||
|
ReadWriteBufferFromHTTP::HTTPHeaderEntries{});
|
||||||
|
|
||||||
|
auto input_stream = context.getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE);
|
||||||
|
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(read_buf_ptr));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
88
src/Bridge/LibraryBridgeHelper.h
Normal file
88
src/Bridge/LibraryBridgeHelper.h
Normal file
@ -0,0 +1,88 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||||
|
#include <Poco/Logger.h>
|
||||||
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
|
#include <Poco/URI.h>
|
||||||
|
#include <Bridge/IBridgeHelper.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class LibraryBridgeHelper : public IBridgeHelper
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
static constexpr inline size_t DEFAULT_PORT = 9012;
|
||||||
|
|
||||||
|
LibraryBridgeHelper(const Context & context_, const Block & sample_block, const Field & dictionary_id_);
|
||||||
|
|
||||||
|
bool initLibrary(const std::string & library_path, const std::string library_settings, const std::string attributes_names);
|
||||||
|
|
||||||
|
bool cloneLibrary(const Field & other_dictionary_id);
|
||||||
|
|
||||||
|
bool removeLibrary();
|
||||||
|
|
||||||
|
bool isModified();
|
||||||
|
|
||||||
|
bool supportsSelectiveLoad();
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadAll();
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadIds(const std::string ids_string);
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadKeys(const Block & requested_block);
|
||||||
|
|
||||||
|
BlockInputStreamPtr loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {});
|
||||||
|
|
||||||
|
bool executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {});
|
||||||
|
|
||||||
|
|
||||||
|
protected:
|
||||||
|
void startBridge(std::unique_ptr<ShellCommand> cmd) const override;
|
||||||
|
|
||||||
|
const String serviceAlias() const override { return "clickhouse-library-bridge"; }
|
||||||
|
|
||||||
|
const String serviceFileName() const override { return serviceAlias(); }
|
||||||
|
|
||||||
|
size_t getDefaultPort() const override { return DEFAULT_PORT; }
|
||||||
|
|
||||||
|
bool startBridgeManually() const override { return false; }
|
||||||
|
|
||||||
|
const String configPrefix() const override { return "library_bridge"; }
|
||||||
|
|
||||||
|
const Context & getContext() const override { return context; }
|
||||||
|
|
||||||
|
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||||
|
|
||||||
|
Poco::Logger * getLog() const override { return log; }
|
||||||
|
|
||||||
|
const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; }
|
||||||
|
|
||||||
|
Poco::URI createBaseURI() const override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
static constexpr inline auto LIB_NEW_METHOD = "libNew";
|
||||||
|
static constexpr inline auto LIB_CLONE_METHOD = "libClone";
|
||||||
|
static constexpr inline auto LIB_DELETE_METHOD = "libDelete";
|
||||||
|
static constexpr inline auto LOAD_ALL_METHOD = "loadAll";
|
||||||
|
static constexpr inline auto LOAD_IDS_METHOD = "loadIds";
|
||||||
|
static constexpr inline auto LOAD_KEYS_METHOD = "loadKeys";
|
||||||
|
static constexpr inline auto IS_MODIFIED_METHOD = "isModified";
|
||||||
|
static constexpr inline auto SUPPORTS_SELECTIVE_LOAD_METHOD = "supportsSelectiveLoad";
|
||||||
|
|
||||||
|
Poco::URI createRequestURI(const String & method) const;
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
const Context & context;
|
||||||
|
const Block sample_block;
|
||||||
|
const Poco::Util::AbstractConfiguration & config;
|
||||||
|
const Poco::Timespan http_timeout;
|
||||||
|
|
||||||
|
Field dictionary_id;
|
||||||
|
std::string bridge_host;
|
||||||
|
size_t bridge_port;
|
||||||
|
};
|
||||||
|
}
|
266
src/Bridge/XDBCBridgeHelper.h
Normal file
266
src/Bridge/XDBCBridgeHelper.h
Normal file
@ -0,0 +1,266 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Access/AccessType.h>
|
||||||
|
#include <Parsers/IdentifierQuotingStyle.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
#include <Poco/Logger.h>
|
||||||
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
|
#include <Poco/Path.h>
|
||||||
|
#include <Poco/URI.h>
|
||||||
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <Common/ShellCommand.h>
|
||||||
|
#include <IO/ConnectionTimeoutsContext.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <ext/range.h>
|
||||||
|
#include <Bridge/IBridgeHelper.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Class for Helpers for XDBC-bridges, provide utility methods, not main request.
|
||||||
|
class IXDBCBridgeHelper : public IBridgeHelper
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
virtual std::vector<std::pair<std::string, std::string>> getURLParams(const std::string & cols, UInt64 max_block_size) const = 0;
|
||||||
|
|
||||||
|
virtual Poco::URI getColumnsInfoURI() const = 0;
|
||||||
|
|
||||||
|
virtual IdentifierQuotingStyle getIdentifierQuotingStyle() = 0;
|
||||||
|
|
||||||
|
virtual bool isSchemaAllowed() = 0;
|
||||||
|
|
||||||
|
virtual const String getName() const = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
using BridgeHelperPtr = std::shared_ptr<IXDBCBridgeHelper>;
|
||||||
|
|
||||||
|
|
||||||
|
template <typename BridgeHelperMixin>
|
||||||
|
class XDBCBridgeHelper : public IXDBCBridgeHelper
|
||||||
|
{
|
||||||
|
|
||||||
|
public:
|
||||||
|
static constexpr inline auto DEFAULT_PORT = BridgeHelperMixin::DEFAULT_PORT;
|
||||||
|
static constexpr inline auto COL_INFO_HANDLER = "/columns_info";
|
||||||
|
static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote";
|
||||||
|
static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed";
|
||||||
|
|
||||||
|
XDBCBridgeHelper(
|
||||||
|
const Context & global_context_,
|
||||||
|
const Poco::Timespan & http_timeout_,
|
||||||
|
const std::string & connection_string_)
|
||||||
|
: log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
|
||||||
|
, connection_string(connection_string_)
|
||||||
|
, http_timeout(http_timeout_)
|
||||||
|
, context(global_context_)
|
||||||
|
, config(context.getConfigRef())
|
||||||
|
{
|
||||||
|
bridge_host = config.getString(BridgeHelperMixin::configPrefix() + ".host", DEFAULT_HOST);
|
||||||
|
bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
protected:
|
||||||
|
auto getConnectionString() const { return connection_string; }
|
||||||
|
|
||||||
|
const String getName() const override { return BridgeHelperMixin::getName(); }
|
||||||
|
|
||||||
|
size_t getDefaultPort() const override { return DEFAULT_PORT; }
|
||||||
|
|
||||||
|
const String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); }
|
||||||
|
|
||||||
|
/// Same for odbc and jdbc
|
||||||
|
const String serviceFileName() const override { return "clickhouse-odbc-bridge"; }
|
||||||
|
|
||||||
|
const String configPrefix() const override { return BridgeHelperMixin::configPrefix(); }
|
||||||
|
|
||||||
|
const Context & getContext() const override { return context; }
|
||||||
|
|
||||||
|
const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; }
|
||||||
|
|
||||||
|
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||||
|
|
||||||
|
Poco::Logger * getLog() const override { return log; }
|
||||||
|
|
||||||
|
bool startBridgeManually() const override { return BridgeHelperMixin::startBridgeManually(); }
|
||||||
|
|
||||||
|
Poco::URI createBaseURI() const override
|
||||||
|
{
|
||||||
|
Poco::URI uri;
|
||||||
|
uri.setHost(bridge_host);
|
||||||
|
uri.setPort(bridge_port);
|
||||||
|
uri.setScheme("http");
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
void startBridge(std::unique_ptr<ShellCommand> cmd) const override
|
||||||
|
{
|
||||||
|
context.addBridgeCommand(std::move(cmd));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private:
|
||||||
|
using Configuration = Poco::Util::AbstractConfiguration;
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
std::string connection_string;
|
||||||
|
const Poco::Timespan & http_timeout;
|
||||||
|
std::string bridge_host;
|
||||||
|
size_t bridge_port;
|
||||||
|
|
||||||
|
const Context & context;
|
||||||
|
const Configuration & config;
|
||||||
|
|
||||||
|
std::optional<IdentifierQuotingStyle> quote_style;
|
||||||
|
std::optional<bool> is_schema_allowed;
|
||||||
|
|
||||||
|
|
||||||
|
protected:
|
||||||
|
using URLParams = std::vector<std::pair<std::string, std::string>>;
|
||||||
|
|
||||||
|
Poco::URI getColumnsInfoURI() const override
|
||||||
|
{
|
||||||
|
auto uri = createBaseURI();
|
||||||
|
uri.setPath(COL_INFO_HANDLER);
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
URLParams getURLParams(const std::string & cols, UInt64 max_block_size) const override
|
||||||
|
{
|
||||||
|
std::vector<std::pair<std::string, std::string>> result;
|
||||||
|
|
||||||
|
result.emplace_back("connection_string", connection_string); /// already validated
|
||||||
|
result.emplace_back("columns", cols);
|
||||||
|
result.emplace_back("max_block_size", std::to_string(max_block_size));
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool isSchemaAllowed() override
|
||||||
|
{
|
||||||
|
if (!is_schema_allowed.has_value())
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
|
||||||
|
auto uri = createBaseURI();
|
||||||
|
uri.setPath(SCHEMA_ALLOWED_HANDLER);
|
||||||
|
uri.addQueryParameter("connection_string", getConnectionString());
|
||||||
|
|
||||||
|
ReadWriteBufferFromHTTP buf(
|
||||||
|
uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
||||||
|
|
||||||
|
bool res;
|
||||||
|
readBoolText(res, buf);
|
||||||
|
is_schema_allowed = res;
|
||||||
|
}
|
||||||
|
|
||||||
|
return *is_schema_allowed;
|
||||||
|
}
|
||||||
|
|
||||||
|
IdentifierQuotingStyle getIdentifierQuotingStyle() override
|
||||||
|
{
|
||||||
|
if (!quote_style.has_value())
|
||||||
|
{
|
||||||
|
startBridgeSync();
|
||||||
|
|
||||||
|
auto uri = createBaseURI();
|
||||||
|
uri.setPath(IDENTIFIER_QUOTE_HANDLER);
|
||||||
|
uri.addQueryParameter("connection_string", getConnectionString());
|
||||||
|
|
||||||
|
ReadWriteBufferFromHTTP buf(
|
||||||
|
uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
||||||
|
std::string character;
|
||||||
|
readStringBinary(character, buf);
|
||||||
|
if (character.length() > 1)
|
||||||
|
throw Exception("Failed to parse quoting style from '" + character + "' for service " + BridgeHelperMixin::serviceAlias(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
else if (character.length() == 0)
|
||||||
|
quote_style = IdentifierQuotingStyle::None;
|
||||||
|
else if (character[0] == '`')
|
||||||
|
quote_style = IdentifierQuotingStyle::Backticks;
|
||||||
|
else if (character[0] == '"')
|
||||||
|
quote_style = IdentifierQuotingStyle::DoubleQuotes;
|
||||||
|
else
|
||||||
|
throw Exception("Can not map quote identifier '" + character + "' to enum value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
return *quote_style;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct JDBCBridgeMixin
|
||||||
|
{
|
||||||
|
static constexpr inline auto DEFAULT_PORT = 9019;
|
||||||
|
|
||||||
|
static const String configPrefix()
|
||||||
|
{
|
||||||
|
return "jdbc_bridge";
|
||||||
|
}
|
||||||
|
|
||||||
|
static const String serviceAlias()
|
||||||
|
{
|
||||||
|
return "clickhouse-jdbc-bridge";
|
||||||
|
}
|
||||||
|
|
||||||
|
static const String getName()
|
||||||
|
{
|
||||||
|
return "JDBC";
|
||||||
|
}
|
||||||
|
|
||||||
|
static AccessType getSourceAccessType()
|
||||||
|
{
|
||||||
|
return AccessType::JDBC;
|
||||||
|
}
|
||||||
|
|
||||||
|
static bool startBridgeManually()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct ODBCBridgeMixin
|
||||||
|
{
|
||||||
|
static constexpr inline auto DEFAULT_PORT = 9018;
|
||||||
|
|
||||||
|
static const String configPrefix()
|
||||||
|
{
|
||||||
|
return "odbc_bridge";
|
||||||
|
}
|
||||||
|
|
||||||
|
static const String serviceAlias()
|
||||||
|
{
|
||||||
|
return "clickhouse-odbc-bridge";
|
||||||
|
}
|
||||||
|
|
||||||
|
static const String getName()
|
||||||
|
{
|
||||||
|
return "ODBC";
|
||||||
|
}
|
||||||
|
|
||||||
|
static AccessType getSourceAccessType()
|
||||||
|
{
|
||||||
|
return AccessType::ODBC;
|
||||||
|
}
|
||||||
|
|
||||||
|
static bool startBridgeManually()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
17
src/Bridge/ya.make
Normal file
17
src/Bridge/ya.make
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it.
|
||||||
|
OWNER(g:clickhouse)
|
||||||
|
|
||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
IBridgeHelper.cpp
|
||||||
|
LibraryBridgeHelper.cpp
|
||||||
|
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
14
src/Bridge/ya.make.in
Normal file
14
src/Bridge/ya.make.in
Normal file
@ -0,0 +1,14 @@
|
|||||||
|
OWNER(g:clickhouse)
|
||||||
|
|
||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
<? find . -name '*.cpp' | grep -v -F tests | sed 's/^\.\// /' | sort ?>
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -190,6 +190,7 @@ add_object_library(clickhouse_storages_distributed Storages/Distributed)
|
|||||||
add_object_library(clickhouse_storages_mergetree Storages/MergeTree)
|
add_object_library(clickhouse_storages_mergetree Storages/MergeTree)
|
||||||
add_object_library(clickhouse_storages_liveview Storages/LiveView)
|
add_object_library(clickhouse_storages_liveview Storages/LiveView)
|
||||||
add_object_library(clickhouse_client Client)
|
add_object_library(clickhouse_client Client)
|
||||||
|
add_object_library(clickhouse_bridge Bridge)
|
||||||
add_object_library(clickhouse_server Server)
|
add_object_library(clickhouse_server Server)
|
||||||
add_object_library(clickhouse_server_http Server/HTTP)
|
add_object_library(clickhouse_server_http Server/HTTP)
|
||||||
add_object_library(clickhouse_formats Formats)
|
add_object_library(clickhouse_formats Formats)
|
||||||
|
@ -590,7 +590,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
void increment(ErrorCode error_code, bool remote, const std::string & message, const FramePointers & trace)
|
void increment(ErrorCode error_code, bool remote, const std::string & message, const FramePointers & trace)
|
||||||
{
|
{
|
||||||
if (error_code >= end())
|
if (error_code < 0 || error_code >= end())
|
||||||
{
|
{
|
||||||
/// For everything outside the range, use END.
|
/// For everything outside the range, use END.
|
||||||
/// (end() is the pointer pass the end, while END is the last value that has an element in values array).
|
/// (end() is the pointer pass the end, while END is the last value that has an element in values array).
|
||||||
|
@ -1,351 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <IO/ReadHelpers.h>
|
|
||||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Access/AccessType.h>
|
|
||||||
#include <Parsers/IdentifierQuotingStyle.h>
|
|
||||||
#include <Poco/File.h>
|
|
||||||
#include <Poco/Logger.h>
|
|
||||||
#include <Poco/Net/HTTPRequest.h>
|
|
||||||
#include <Poco/Path.h>
|
|
||||||
#include <Poco/URI.h>
|
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
|
||||||
#include <Common/ShellCommand.h>
|
|
||||||
#include <IO/ConnectionTimeoutsContext.h>
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
#include <ext/range.h>
|
|
||||||
|
|
||||||
#if !defined(ARCADIA_BUILD)
|
|
||||||
# include <Common/config.h>
|
|
||||||
#endif
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING;
|
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Class for Helpers for Xdbc-bridges, provide utility methods, not main request
|
|
||||||
*/
|
|
||||||
class IXDBCBridgeHelper
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
static constexpr inline auto DEFAULT_FORMAT = "RowBinary";
|
|
||||||
|
|
||||||
virtual std::vector<std::pair<std::string, std::string>> getURLParams(const std::string & cols, UInt64 max_block_size) const = 0;
|
|
||||||
virtual void startBridgeSync() const = 0;
|
|
||||||
virtual Poco::URI getMainURI() const = 0;
|
|
||||||
virtual Poco::URI getColumnsInfoURI() const = 0;
|
|
||||||
virtual IdentifierQuotingStyle getIdentifierQuotingStyle() = 0;
|
|
||||||
virtual bool isSchemaAllowed() = 0;
|
|
||||||
virtual String getName() const = 0;
|
|
||||||
|
|
||||||
virtual ~IXDBCBridgeHelper() = default;
|
|
||||||
};
|
|
||||||
|
|
||||||
using BridgeHelperPtr = std::shared_ptr<IXDBCBridgeHelper>;
|
|
||||||
|
|
||||||
template <typename BridgeHelperMixin>
|
|
||||||
class XDBCBridgeHelper : public IXDBCBridgeHelper
|
|
||||||
{
|
|
||||||
private:
|
|
||||||
Poco::Timespan http_timeout;
|
|
||||||
|
|
||||||
std::string connection_string;
|
|
||||||
|
|
||||||
Poco::URI ping_url;
|
|
||||||
|
|
||||||
Poco::Logger * log = &Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper");
|
|
||||||
|
|
||||||
std::optional<IdentifierQuotingStyle> quote_style;
|
|
||||||
std::optional<bool> is_schema_allowed;
|
|
||||||
|
|
||||||
protected:
|
|
||||||
auto getConnectionString() const
|
|
||||||
{
|
|
||||||
return connection_string;
|
|
||||||
}
|
|
||||||
|
|
||||||
public:
|
|
||||||
using Configuration = Poco::Util::AbstractConfiguration;
|
|
||||||
|
|
||||||
const Context & context;
|
|
||||||
const Configuration & config;
|
|
||||||
|
|
||||||
static constexpr inline auto DEFAULT_HOST = "127.0.0.1";
|
|
||||||
static constexpr inline auto DEFAULT_PORT = BridgeHelperMixin::DEFAULT_PORT;
|
|
||||||
static constexpr inline auto PING_HANDLER = "/ping";
|
|
||||||
static constexpr inline auto MAIN_HANDLER = "/";
|
|
||||||
static constexpr inline auto COL_INFO_HANDLER = "/columns_info";
|
|
||||||
static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote";
|
|
||||||
static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed";
|
|
||||||
static constexpr inline auto PING_OK_ANSWER = "Ok.";
|
|
||||||
|
|
||||||
XDBCBridgeHelper(const Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_)
|
|
||||||
: http_timeout(http_timeout_), connection_string(connection_string_), context(global_context_), config(context.getConfigRef())
|
|
||||||
{
|
|
||||||
size_t bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT);
|
|
||||||
std::string bridge_host = config.getString(BridgeHelperMixin::configPrefix() + ".host", DEFAULT_HOST);
|
|
||||||
|
|
||||||
ping_url.setHost(bridge_host);
|
|
||||||
ping_url.setPort(bridge_port);
|
|
||||||
ping_url.setScheme("http");
|
|
||||||
ping_url.setPath(PING_HANDLER);
|
|
||||||
}
|
|
||||||
|
|
||||||
String getName() const override
|
|
||||||
{
|
|
||||||
return BridgeHelperMixin::getName();
|
|
||||||
}
|
|
||||||
|
|
||||||
IdentifierQuotingStyle getIdentifierQuotingStyle() override
|
|
||||||
{
|
|
||||||
if (!quote_style.has_value())
|
|
||||||
{
|
|
||||||
startBridgeSync();
|
|
||||||
|
|
||||||
auto uri = createBaseURI();
|
|
||||||
uri.setPath(IDENTIFIER_QUOTE_HANDLER);
|
|
||||||
uri.addQueryParameter("connection_string", getConnectionString());
|
|
||||||
|
|
||||||
ReadWriteBufferFromHTTP buf(
|
|
||||||
uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
|
||||||
std::string character;
|
|
||||||
readStringBinary(character, buf);
|
|
||||||
if (character.length() > 1)
|
|
||||||
throw Exception("Failed to parse quoting style from '" + character + "' for service " + BridgeHelperMixin::serviceAlias(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
||||||
else if (character.length() == 0)
|
|
||||||
quote_style = IdentifierQuotingStyle::None;
|
|
||||||
else if (character[0] == '`')
|
|
||||||
quote_style = IdentifierQuotingStyle::Backticks;
|
|
||||||
else if (character[0] == '"')
|
|
||||||
quote_style = IdentifierQuotingStyle::DoubleQuotes;
|
|
||||||
else
|
|
||||||
throw Exception("Can not map quote identifier '" + character + "' to enum value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
||||||
}
|
|
||||||
|
|
||||||
return *quote_style;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool isSchemaAllowed() override
|
|
||||||
{
|
|
||||||
if (!is_schema_allowed.has_value())
|
|
||||||
{
|
|
||||||
startBridgeSync();
|
|
||||||
|
|
||||||
auto uri = createBaseURI();
|
|
||||||
uri.setPath(SCHEMA_ALLOWED_HANDLER);
|
|
||||||
uri.addQueryParameter("connection_string", getConnectionString());
|
|
||||||
|
|
||||||
ReadWriteBufferFromHTTP buf(
|
|
||||||
uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
|
||||||
|
|
||||||
bool res;
|
|
||||||
readBoolText(res, buf);
|
|
||||||
is_schema_allowed = res;
|
|
||||||
}
|
|
||||||
|
|
||||||
return *is_schema_allowed;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @todo leaky abstraction - used by external API's
|
|
||||||
*/
|
|
||||||
std::vector<std::pair<std::string, std::string>> getURLParams(const std::string & cols, UInt64 max_block_size) const override
|
|
||||||
{
|
|
||||||
std::vector<std::pair<std::string, std::string>> result;
|
|
||||||
|
|
||||||
result.emplace_back("connection_string", connection_string); /// already validated
|
|
||||||
result.emplace_back("columns", cols);
|
|
||||||
result.emplace_back("max_block_size", std::to_string(max_block_size));
|
|
||||||
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Performs spawn of external daemon
|
|
||||||
*/
|
|
||||||
void startBridgeSync() const override
|
|
||||||
{
|
|
||||||
if (!checkBridgeIsRunning())
|
|
||||||
{
|
|
||||||
LOG_TRACE(log, "{} is not running, will try to start it", BridgeHelperMixin::serviceAlias());
|
|
||||||
startBridge();
|
|
||||||
bool started = false;
|
|
||||||
|
|
||||||
uint64_t milliseconds_to_wait = 10; /// Exponential backoff
|
|
||||||
uint64_t counter = 0;
|
|
||||||
while (milliseconds_to_wait < 10000)
|
|
||||||
{
|
|
||||||
++counter;
|
|
||||||
LOG_TRACE(log, "Checking {} is running, try {}", BridgeHelperMixin::serviceAlias(), counter);
|
|
||||||
if (checkBridgeIsRunning())
|
|
||||||
{
|
|
||||||
started = true;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(milliseconds_to_wait));
|
|
||||||
milliseconds_to_wait *= 2;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!started)
|
|
||||||
throw Exception(BridgeHelperMixin::getName() + "BridgeHelper: " + BridgeHelperMixin::serviceAlias() + " is not responding",
|
|
||||||
ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* URI to fetch the data from external service
|
|
||||||
*/
|
|
||||||
Poco::URI getMainURI() const override
|
|
||||||
{
|
|
||||||
auto uri = createBaseURI();
|
|
||||||
uri.setPath(MAIN_HANDLER);
|
|
||||||
return uri;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* URI to retrieve column description from external service
|
|
||||||
*/
|
|
||||||
Poco::URI getColumnsInfoURI() const override
|
|
||||||
{
|
|
||||||
auto uri = createBaseURI();
|
|
||||||
uri.setPath(COL_INFO_HANDLER);
|
|
||||||
return uri;
|
|
||||||
}
|
|
||||||
|
|
||||||
protected:
|
|
||||||
Poco::URI createBaseURI() const
|
|
||||||
{
|
|
||||||
Poco::URI uri;
|
|
||||||
uri.setHost(ping_url.getHost());
|
|
||||||
uri.setPort(ping_url.getPort());
|
|
||||||
uri.setScheme("http");
|
|
||||||
return uri;
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
bool checkBridgeIsRunning() const
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
ReadWriteBufferFromHTTP buf(
|
|
||||||
ping_url, Poco::Net::HTTPRequest::HTTP_GET, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
|
||||||
return checkString(XDBCBridgeHelper::PING_OK_ANSWER, buf);
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/* Contains logic for instantiation of the bridge instance */
|
|
||||||
void startBridge() const
|
|
||||||
{
|
|
||||||
auto cmd = BridgeHelperMixin::startBridge(config, log, http_timeout);
|
|
||||||
context.addXDBCBridgeCommand(std::move(cmd));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
struct JDBCBridgeMixin
|
|
||||||
{
|
|
||||||
static constexpr inline auto DEFAULT_PORT = 9019;
|
|
||||||
static const String configPrefix()
|
|
||||||
{
|
|
||||||
return "jdbc_bridge";
|
|
||||||
}
|
|
||||||
static const String serviceAlias()
|
|
||||||
{
|
|
||||||
return "clickhouse-jdbc-bridge";
|
|
||||||
}
|
|
||||||
static const String getName()
|
|
||||||
{
|
|
||||||
return "JDBC";
|
|
||||||
}
|
|
||||||
static AccessType getSourceAccessType()
|
|
||||||
{
|
|
||||||
return AccessType::JDBC;
|
|
||||||
}
|
|
||||||
|
|
||||||
static std::unique_ptr<ShellCommand> startBridge(const Poco::Util::AbstractConfiguration &, const Poco::Logger *, const Poco::Timespan &)
|
|
||||||
{
|
|
||||||
throw Exception("jdbc-bridge is not running. Please, start it manually", ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
struct ODBCBridgeMixin
|
|
||||||
{
|
|
||||||
static constexpr inline auto DEFAULT_PORT = 9018;
|
|
||||||
|
|
||||||
static const String configPrefix()
|
|
||||||
{
|
|
||||||
return "odbc_bridge";
|
|
||||||
}
|
|
||||||
static const String serviceAlias()
|
|
||||||
{
|
|
||||||
return "clickhouse-odbc-bridge";
|
|
||||||
}
|
|
||||||
static const String getName()
|
|
||||||
{
|
|
||||||
return "ODBC";
|
|
||||||
}
|
|
||||||
static AccessType getSourceAccessType()
|
|
||||||
{
|
|
||||||
return AccessType::ODBC;
|
|
||||||
}
|
|
||||||
|
|
||||||
static std::unique_ptr<ShellCommand> startBridge(
|
|
||||||
const Poco::Util::AbstractConfiguration & config, Poco::Logger * log, const Poco::Timespan & http_timeout)
|
|
||||||
{
|
|
||||||
/// Path to executable folder
|
|
||||||
Poco::Path path{config.getString("application.dir", "/usr/bin")};
|
|
||||||
|
|
||||||
std::vector<std::string> cmd_args;
|
|
||||||
path.setFileName("clickhouse-odbc-bridge");
|
|
||||||
|
|
||||||
#if !CLICKHOUSE_SPLIT_BINARY
|
|
||||||
cmd_args.push_back("odbc-bridge");
|
|
||||||
#endif
|
|
||||||
|
|
||||||
cmd_args.push_back("--http-port");
|
|
||||||
cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", DEFAULT_PORT)));
|
|
||||||
cmd_args.push_back("--listen-host");
|
|
||||||
cmd_args.push_back(config.getString(configPrefix() + ".listen_host", XDBCBridgeHelper<ODBCBridgeMixin>::DEFAULT_HOST));
|
|
||||||
cmd_args.push_back("--http-timeout");
|
|
||||||
cmd_args.push_back(std::to_string(http_timeout.totalMicroseconds()));
|
|
||||||
if (config.has("logger." + configPrefix() + "_log"))
|
|
||||||
{
|
|
||||||
cmd_args.push_back("--log-path");
|
|
||||||
cmd_args.push_back(config.getString("logger." + configPrefix() + "_log"));
|
|
||||||
}
|
|
||||||
if (config.has("logger." + configPrefix() + "_errlog"))
|
|
||||||
{
|
|
||||||
cmd_args.push_back("--err-log-path");
|
|
||||||
cmd_args.push_back(config.getString("logger." + configPrefix() + "_errlog"));
|
|
||||||
}
|
|
||||||
if (config.has("logger." + configPrefix() + "_stdout"))
|
|
||||||
{
|
|
||||||
cmd_args.push_back("--stdout-path");
|
|
||||||
cmd_args.push_back(config.getString("logger." + configPrefix() + "_stdout"));
|
|
||||||
}
|
|
||||||
if (config.has("logger." + configPrefix() + "_stderr"))
|
|
||||||
{
|
|
||||||
cmd_args.push_back("--stderr-path");
|
|
||||||
cmd_args.push_back(config.getString("logger." + configPrefix() + "_stderr"));
|
|
||||||
}
|
|
||||||
if (config.has("logger." + configPrefix() + "_level"))
|
|
||||||
{
|
|
||||||
cmd_args.push_back("--log-level");
|
|
||||||
cmd_args.push_back(config.getString("logger." + configPrefix() + "_level"));
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG_TRACE(log, "Starting {}", serviceAlias());
|
|
||||||
|
|
||||||
return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
@ -13,7 +13,10 @@ namespace DB
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static void formatReadable(double size, DB::WriteBuffer & out, int precision, const char ** units, size_t units_size, double delimiter)
|
// I wanted to make this ALWAYS_INLINE to prevent flappy performance tests,
|
||||||
|
// but GCC complains it may not be inlined.
|
||||||
|
static void formatReadable(double size, DB::WriteBuffer & out,
|
||||||
|
int precision, const char ** units, size_t units_size, double delimiter)
|
||||||
{
|
{
|
||||||
size_t i = 0;
|
size_t i = 0;
|
||||||
for (; i + 1 < units_size && fabs(size) >= delimiter; ++i)
|
for (; i + 1 < units_size && fabs(size) >= delimiter; ++i)
|
||||||
|
@ -1,6 +1,9 @@
|
|||||||
#include "parseRemoteDescription.h"
|
#include "parseRemoteDescription.h"
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -167,4 +170,27 @@ std::vector<String> parseRemoteDescription(const String & description, size_t l,
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
std::vector<std::pair<String, uint16_t>> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port)
|
||||||
|
{
|
||||||
|
auto addresses = parseRemoteDescription(description, 0, description.size(), '|', max_addresses);
|
||||||
|
std::vector<std::pair<String, uint16_t>> result;
|
||||||
|
|
||||||
|
for (const auto & address : addresses)
|
||||||
|
{
|
||||||
|
size_t colon = address.find(':');
|
||||||
|
if (colon == String::npos)
|
||||||
|
{
|
||||||
|
LOG_WARNING(&Poco::Logger::get("ParseRemoteDescription"), "Port is not found for host: {}. Using default port {}", default_port);
|
||||||
|
result.emplace_back(std::make_pair(address, default_port));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
result.emplace_back(std::make_pair(address.substr(0, colon), DB::parseFromString<UInt16>(address.substr(colon + 1))));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -17,4 +17,7 @@ namespace DB
|
|||||||
*/
|
*/
|
||||||
std::vector<String> parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses);
|
std::vector<String> parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses);
|
||||||
|
|
||||||
|
/// Parse remote description for external database (MySQL or PostgreSQL).
|
||||||
|
std::vector<std::pair<String, uint16_t>> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -371,6 +371,7 @@ class IColumn;
|
|||||||
\
|
\
|
||||||
M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \
|
M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \
|
||||||
M(Int64, postgresql_connection_pool_wait_timeout, -1, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \
|
M(Int64, postgresql_connection_pool_wait_timeout, -1, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \
|
||||||
|
M(UInt64, glob_expansion_max_elements, 1000, "Maximum number of allowed addresses (For external storages, table functions, etc).", 0) \
|
||||||
\
|
\
|
||||||
M(Seconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \
|
M(Seconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \
|
||||||
M(UInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \
|
M(UInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \
|
||||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
PostgreSQLBlockInputStream::PostgreSQLBlockInputStream(
|
PostgreSQLBlockInputStream::PostgreSQLBlockInputStream(
|
||||||
PostgreSQLConnectionHolderPtr connection_,
|
postgres::ConnectionHolderPtr connection_,
|
||||||
const std::string & query_str_,
|
const std::string & query_str_,
|
||||||
const Block & sample_block,
|
const Block & sample_block,
|
||||||
const UInt64 max_block_size_)
|
const UInt64 max_block_size_)
|
||||||
@ -120,8 +120,15 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view
|
|||||||
switch (type)
|
switch (type)
|
||||||
{
|
{
|
||||||
case ValueType::vtUInt8:
|
case ValueType::vtUInt8:
|
||||||
|
{
|
||||||
|
if (value == "t")
|
||||||
|
assert_cast<ColumnUInt8 &>(column).insertValue(1);
|
||||||
|
else if (value == "f")
|
||||||
|
assert_cast<ColumnUInt8 &>(column).insertValue(0);
|
||||||
|
else
|
||||||
assert_cast<ColumnUInt8 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
assert_cast<ColumnUInt8 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
||||||
break;
|
break;
|
||||||
|
}
|
||||||
case ValueType::vtUInt16:
|
case ValueType::vtUInt16:
|
||||||
assert_cast<ColumnUInt16 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
assert_cast<ColumnUInt16 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
||||||
break;
|
break;
|
||||||
|
@ -19,7 +19,7 @@ class PostgreSQLBlockInputStream : public IBlockInputStream
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
PostgreSQLBlockInputStream(
|
PostgreSQLBlockInputStream(
|
||||||
PostgreSQLConnectionHolderPtr connection_,
|
postgres::ConnectionHolderPtr connection_,
|
||||||
const std::string & query_str,
|
const std::string & query_str,
|
||||||
const Block & sample_block,
|
const Block & sample_block,
|
||||||
const UInt64 max_block_size_);
|
const UInt64 max_block_size_);
|
||||||
@ -46,7 +46,7 @@ private:
|
|||||||
const UInt64 max_block_size;
|
const UInt64 max_block_size;
|
||||||
ExternalResultDescription description;
|
ExternalResultDescription description;
|
||||||
|
|
||||||
PostgreSQLConnectionHolderPtr connection;
|
postgres::ConnectionHolderPtr connection;
|
||||||
std::unique_ptr<pqxx::read_transaction> tx;
|
std::unique_ptr<pqxx::read_transaction> tx;
|
||||||
std::unique_ptr<pqxx::stream_from> stream;
|
std::unique_ptr<pqxx::stream_from> stream;
|
||||||
|
|
||||||
|
@ -30,6 +30,7 @@
|
|||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_MYSQL || USE_LIBPQXX
|
#if USE_MYSQL || USE_LIBPQXX
|
||||||
|
#include <Common/parseRemoteDescription.h>
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Common/parseAddress.h>
|
#include <Common/parseAddress.h>
|
||||||
#endif
|
#endif
|
||||||
@ -133,19 +134,20 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
|||||||
ASTs & arguments = engine->arguments->children;
|
ASTs & arguments = engine->arguments->children;
|
||||||
arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context);
|
arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context);
|
||||||
|
|
||||||
const auto & host_name_and_port = safeGetLiteralValue<String>(arguments[0], engine_name);
|
const auto & host_port = safeGetLiteralValue<String>(arguments[0], engine_name);
|
||||||
const auto & mysql_database_name = safeGetLiteralValue<String>(arguments[1], engine_name);
|
const auto & mysql_database_name = safeGetLiteralValue<String>(arguments[1], engine_name);
|
||||||
const auto & mysql_user_name = safeGetLiteralValue<String>(arguments[2], engine_name);
|
const auto & mysql_user_name = safeGetLiteralValue<String>(arguments[2], engine_name);
|
||||||
const auto & mysql_user_password = safeGetLiteralValue<String>(arguments[3], engine_name);
|
const auto & mysql_user_password = safeGetLiteralValue<String>(arguments[3], engine_name);
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306);
|
|
||||||
auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port);
|
|
||||||
|
|
||||||
if (engine_name == "MySQL")
|
if (engine_name == "MySQL")
|
||||||
{
|
{
|
||||||
auto mysql_database_settings = std::make_unique<ConnectionMySQLSettings>();
|
auto mysql_database_settings = std::make_unique<ConnectionMySQLSettings>();
|
||||||
|
/// Split into replicas if needed.
|
||||||
|
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
|
||||||
|
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
|
||||||
|
auto mysql_pool = mysqlxx::PoolWithFailover(mysql_database_name, addresses, mysql_user_name, mysql_user_password);
|
||||||
|
|
||||||
mysql_database_settings->loadFromQueryContext(context);
|
mysql_database_settings->loadFromQueryContext(context);
|
||||||
mysql_database_settings->loadFromQuery(*engine_define); /// higher priority
|
mysql_database_settings->loadFromQuery(*engine_define); /// higher priority
|
||||||
@ -154,7 +156,10 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
|||||||
context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_database_settings), std::move(mysql_pool));
|
context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_database_settings), std::move(mysql_pool));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
const auto & [remote_host_name, remote_port] = parseAddress(host_port, 3306);
|
||||||
MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password);
|
MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password);
|
||||||
|
auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password);
|
||||||
|
|
||||||
|
|
||||||
auto materialize_mode_settings = std::make_unique<MaterializeMySQLSettings>();
|
auto materialize_mode_settings = std::make_unique<MaterializeMySQLSettings>();
|
||||||
|
|
||||||
@ -243,12 +248,14 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
|||||||
if (engine->arguments->children.size() == 5)
|
if (engine->arguments->children.size() == 5)
|
||||||
use_table_cache = safeGetLiteralValue<UInt64>(engine_args[4], engine_name);
|
use_table_cache = safeGetLiteralValue<UInt64>(engine_args[4], engine_name);
|
||||||
|
|
||||||
auto parsed_host_port = parseAddress(host_port, 5432);
|
/// Split into replicas if needed.
|
||||||
|
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
|
||||||
|
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
|
||||||
|
|
||||||
/// no connection is made here
|
/// no connection is made here
|
||||||
auto connection_pool = std::make_shared<PostgreSQLConnectionPool>(
|
auto connection_pool = std::make_shared<postgres::PoolWithFailover>(
|
||||||
postgres_database_name,
|
postgres_database_name,
|
||||||
parsed_host_port.first, parsed_host_port.second,
|
addresses,
|
||||||
username, password,
|
username, password,
|
||||||
context.getSettingsRef().postgresql_connection_pool_size,
|
context.getSettingsRef().postgresql_connection_pool_size,
|
||||||
context.getSettingsRef().postgresql_connection_pool_wait_timeout);
|
context.getSettingsRef().postgresql_connection_pool_wait_timeout);
|
||||||
|
@ -45,8 +45,14 @@ constexpr static const auto suffix = ".remove_flag";
|
|||||||
static constexpr const std::chrono::seconds cleaner_sleep_time{30};
|
static constexpr const std::chrono::seconds cleaner_sleep_time{30};
|
||||||
static const std::chrono::seconds lock_acquire_timeout{10};
|
static const std::chrono::seconds lock_acquire_timeout{10};
|
||||||
|
|
||||||
DatabaseConnectionMySQL::DatabaseConnectionMySQL(const Context & context, const String & database_name_, const String & metadata_path_,
|
DatabaseConnectionMySQL::DatabaseConnectionMySQL(
|
||||||
const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, std::unique_ptr<ConnectionMySQLSettings> settings_, mysqlxx::Pool && pool)
|
const Context & context,
|
||||||
|
const String & database_name_,
|
||||||
|
const String & metadata_path_,
|
||||||
|
const ASTStorage * database_engine_define_,
|
||||||
|
const String & database_name_in_mysql_,
|
||||||
|
std::unique_ptr<ConnectionMySQLSettings> settings_,
|
||||||
|
mysqlxx::PoolWithFailover && pool)
|
||||||
: IDatabase(database_name_)
|
: IDatabase(database_name_)
|
||||||
, global_context(context.getGlobalContext())
|
, global_context(context.getGlobalContext())
|
||||||
, metadata_path(metadata_path_)
|
, metadata_path(metadata_path_)
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <Databases/DatabasesCommon.h>
|
#include <Databases/DatabasesCommon.h>
|
||||||
#include <Databases/MySQL/ConnectionMySQLSettings.h>
|
#include <Databases/MySQL/ConnectionMySQLSettings.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
|
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
#include <condition_variable>
|
#include <condition_variable>
|
||||||
@ -36,9 +37,13 @@ public:
|
|||||||
~DatabaseConnectionMySQL() override;
|
~DatabaseConnectionMySQL() override;
|
||||||
|
|
||||||
DatabaseConnectionMySQL(
|
DatabaseConnectionMySQL(
|
||||||
const Context & context, const String & database_name, const String & metadata_path,
|
const Context & context,
|
||||||
const ASTStorage * database_engine_define, const String & database_name_in_mysql, std::unique_ptr<ConnectionMySQLSettings> settings_,
|
const String & database_name,
|
||||||
mysqlxx::Pool && pool);
|
const String & metadata_path,
|
||||||
|
const ASTStorage * database_engine_define,
|
||||||
|
const String & database_name_in_mysql,
|
||||||
|
std::unique_ptr<ConnectionMySQLSettings> settings_,
|
||||||
|
mysqlxx::PoolWithFailover && pool);
|
||||||
|
|
||||||
String getEngineName() const override { return "MySQL"; }
|
String getEngineName() const override { return "MySQL"; }
|
||||||
|
|
||||||
@ -91,7 +96,7 @@ private:
|
|||||||
std::atomic<bool> quit{false};
|
std::atomic<bool> quit{false};
|
||||||
std::condition_variable cond;
|
std::condition_variable cond;
|
||||||
|
|
||||||
using MySQLPool = mysqlxx::Pool;
|
using MySQLPool = mysqlxx::PoolWithFailover;
|
||||||
using ModifyTimeAndStorage = std::pair<UInt64, StoragePtr>;
|
using ModifyTimeAndStorage = std::pair<UInt64, StoragePtr>;
|
||||||
|
|
||||||
mutable MySQLPool mysql_pool;
|
mutable MySQLPool mysql_pool;
|
||||||
|
@ -41,7 +41,7 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
std::map<String, NamesAndTypesList> fetchTablesColumnsList(
|
std::map<String, NamesAndTypesList> fetchTablesColumnsList(
|
||||||
mysqlxx::Pool & pool,
|
mysqlxx::PoolWithFailover & pool,
|
||||||
const String & database_name,
|
const String & database_name,
|
||||||
const std::vector<String> & tables_name,
|
const std::vector<String> & tables_name,
|
||||||
bool external_table_functions_use_nulls,
|
bool external_table_functions_use_nulls,
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
#include "config_core.h"
|
#include "config_core.h"
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
|
|
||||||
#include <mysqlxx/Pool.h>
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
|
|
||||||
#include <common/types.h>
|
#include <common/types.h>
|
||||||
#include <Core/MultiEnum.h>
|
#include <Core/MultiEnum.h>
|
||||||
@ -17,7 +17,7 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
std::map<String, NamesAndTypesList> fetchTablesColumnsList(
|
std::map<String, NamesAndTypesList> fetchTablesColumnsList(
|
||||||
mysqlxx::Pool & pool,
|
mysqlxx::PoolWithFailover & pool,
|
||||||
const String & database_name,
|
const String & database_name,
|
||||||
const std::vector<String> & tables_name,
|
const std::vector<String> & tables_name,
|
||||||
bool external_table_functions_use_nulls,
|
bool external_table_functions_use_nulls,
|
||||||
|
@ -40,7 +40,7 @@ DatabasePostgreSQL::DatabasePostgreSQL(
|
|||||||
const ASTStorage * database_engine_define_,
|
const ASTStorage * database_engine_define_,
|
||||||
const String & dbname_,
|
const String & dbname_,
|
||||||
const String & postgres_dbname,
|
const String & postgres_dbname,
|
||||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
postgres::PoolWithFailoverPtr connection_pool_,
|
||||||
const bool cache_tables_)
|
const bool cache_tables_)
|
||||||
: IDatabase(dbname_)
|
: IDatabase(dbname_)
|
||||||
, global_context(context.getGlobalContext())
|
, global_context(context.getGlobalContext())
|
||||||
@ -171,7 +171,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte
|
|||||||
return StoragePtr{};
|
return StoragePtr{};
|
||||||
|
|
||||||
auto storage = StoragePostgreSQL::create(
|
auto storage = StoragePostgreSQL::create(
|
||||||
StorageID(database_name, table_name), table_name, std::make_shared<PostgreSQLConnectionPool>(*connection_pool),
|
StorageID(database_name, table_name), *connection_pool, table_name,
|
||||||
ColumnsDescription{*columns}, ConstraintsDescription{}, context);
|
ColumnsDescription{*columns}, ConstraintsDescription{}, context);
|
||||||
|
|
||||||
if (cache_tables)
|
if (cache_tables)
|
||||||
|
@ -9,14 +9,13 @@
|
|||||||
#include <Databases/DatabasesCommon.h>
|
#include <Databases/DatabasesCommon.h>
|
||||||
#include <Core/BackgroundSchedulePool.h>
|
#include <Core/BackgroundSchedulePool.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class PostgreSQLConnectionPool;
|
|
||||||
using PostgreSQLConnectionPoolPtr = std::shared_ptr<PostgreSQLConnectionPool>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Real-time access to table list and table structure from remote PostgreSQL.
|
/** Real-time access to table list and table structure from remote PostgreSQL.
|
||||||
@ -34,7 +33,7 @@ public:
|
|||||||
const ASTStorage * database_engine_define,
|
const ASTStorage * database_engine_define,
|
||||||
const String & dbname_,
|
const String & dbname_,
|
||||||
const String & postgres_dbname,
|
const String & postgres_dbname,
|
||||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
postgres::PoolWithFailoverPtr connection_pool_,
|
||||||
const bool cache_tables_);
|
const bool cache_tables_);
|
||||||
|
|
||||||
String getEngineName() const override { return "PostgreSQL"; }
|
String getEngineName() const override { return "PostgreSQL"; }
|
||||||
@ -72,7 +71,7 @@ private:
|
|||||||
String metadata_path;
|
String metadata_path;
|
||||||
ASTPtr database_engine_define;
|
ASTPtr database_engine_define;
|
||||||
String dbname;
|
String dbname;
|
||||||
PostgreSQLConnectionPoolPtr connection_pool;
|
postgres::PoolWithFailoverPtr connection_pool;
|
||||||
const bool cache_tables;
|
const bool cache_tables;
|
||||||
|
|
||||||
mutable Tables cached_tables;
|
mutable Tables cached_tables;
|
||||||
|
@ -40,6 +40,8 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl
|
|||||||
res = std::make_shared<DataTypeInt32>();
|
res = std::make_shared<DataTypeInt32>();
|
||||||
else if (type == "bigint")
|
else if (type == "bigint")
|
||||||
res = std::make_shared<DataTypeInt64>();
|
res = std::make_shared<DataTypeInt64>();
|
||||||
|
else if (type == "boolean")
|
||||||
|
res = std::make_shared<DataTypeUInt8>();
|
||||||
else if (type == "real")
|
else if (type == "real")
|
||||||
res = std::make_shared<DataTypeFloat32>();
|
res = std::make_shared<DataTypeFloat32>();
|
||||||
else if (type == "double precision")
|
else if (type == "double precision")
|
||||||
@ -94,7 +96,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl
|
|||||||
|
|
||||||
|
|
||||||
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||||
PostgreSQLConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls)
|
postgres::ConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls)
|
||||||
{
|
{
|
||||||
auto columns = NamesAndTypesList();
|
auto columns = NamesAndTypesList();
|
||||||
|
|
||||||
|
@ -12,7 +12,7 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||||
PostgreSQLConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls);
|
postgres::ConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include "LibraryDictionarySource.h"
|
#include "LibraryDictionarySource.h"
|
||||||
|
|
||||||
#include <DataStreams/OneBlockInputStream.h>
|
#include <DataStreams/OneBlockInputStream.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
@ -6,294 +7,171 @@
|
|||||||
#include <ext/bit_cast.h>
|
#include <ext/bit_cast.h>
|
||||||
#include <ext/range.h>
|
#include <ext/range.h>
|
||||||
#include <ext/scope_guard.h>
|
#include <ext/scope_guard.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
|
||||||
#include "DictionarySourceFactory.h"
|
#include "DictionarySourceFactory.h"
|
||||||
|
#include "DictionarySourceHelpers.h"
|
||||||
#include "DictionaryStructure.h"
|
#include "DictionaryStructure.h"
|
||||||
#include "LibraryDictionarySourceExternal.h"
|
#include "LibraryDictionarySourceExternal.h"
|
||||||
#include "registerDictionaries.h"
|
#include "registerDictionaries.h"
|
||||||
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
|
||||||
extern const int FILE_DOESNT_EXIST;
|
extern const int FILE_DOESNT_EXIST;
|
||||||
extern const int EXTERNAL_LIBRARY_ERROR;
|
extern const int EXTERNAL_LIBRARY_ERROR;
|
||||||
extern const int PATH_ACCESS_DENIED;
|
extern const int PATH_ACCESS_DENIED;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
class CStringsHolder
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
using Container = std::vector<std::string>;
|
|
||||||
explicit CStringsHolder(const Container & strings_pass)
|
|
||||||
{
|
|
||||||
strings_holder = strings_pass;
|
|
||||||
strings.size = strings_holder.size();
|
|
||||||
ptr_holder = std::make_unique<ClickHouseLibrary::CString[]>(strings.size);
|
|
||||||
strings.data = ptr_holder.get();
|
|
||||||
size_t i = 0;
|
|
||||||
for (auto & str : strings_holder)
|
|
||||||
{
|
|
||||||
strings.data[i] = str.c_str();
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
ClickHouseLibrary::CStrings strings; // will pass pointer to lib
|
|
||||||
|
|
||||||
private:
|
|
||||||
std::unique_ptr<ClickHouseLibrary::CString[]> ptr_holder = nullptr;
|
|
||||||
Container strings_holder;
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
constexpr auto lib_config_settings = ".settings";
|
|
||||||
|
|
||||||
|
|
||||||
CStringsHolder getLibSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_root)
|
|
||||||
{
|
|
||||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
||||||
config.keys(config_root, config_keys);
|
|
||||||
CStringsHolder::Container strings;
|
|
||||||
for (const auto & key : config_keys)
|
|
||||||
{
|
|
||||||
std::string key_name = key;
|
|
||||||
auto bracket_pos = key.find('[');
|
|
||||||
if (bracket_pos != std::string::npos && bracket_pos > 0)
|
|
||||||
key_name = key.substr(0, bracket_pos);
|
|
||||||
strings.emplace_back(key_name);
|
|
||||||
strings.emplace_back(config.getString(config_root + "." + key));
|
|
||||||
}
|
|
||||||
return CStringsHolder(strings);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
Block dataToBlock(const Block & sample_block, const ClickHouseLibrary::RawClickHouseLibraryTable data)
|
|
||||||
{
|
|
||||||
if (!data)
|
|
||||||
throw Exception("LibraryDictionarySource: No data returned", ErrorCodes::EXTERNAL_LIBRARY_ERROR);
|
|
||||||
|
|
||||||
const auto * columns_received = static_cast<const ClickHouseLibrary::Table *>(data);
|
|
||||||
if (columns_received->error_code)
|
|
||||||
throw Exception(
|
|
||||||
"LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " "
|
|
||||||
+ (columns_received->error_string ? columns_received->error_string : ""),
|
|
||||||
ErrorCodes::EXTERNAL_LIBRARY_ERROR);
|
|
||||||
|
|
||||||
MutableColumns columns = sample_block.cloneEmptyColumns();
|
|
||||||
|
|
||||||
for (size_t col_n = 0; col_n < columns_received->size; ++col_n)
|
|
||||||
{
|
|
||||||
if (columns.size() != columns_received->data[col_n].size)
|
|
||||||
throw Exception(
|
|
||||||
"LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size)
|
|
||||||
+ ", must be " + std::to_string(columns.size()),
|
|
||||||
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n)
|
|
||||||
{
|
|
||||||
const auto & field = columns_received->data[col_n].data[row_n];
|
|
||||||
if (!field.data)
|
|
||||||
{
|
|
||||||
/// sample_block contains null_value (from config) inside corresponding column
|
|
||||||
const auto & col = sample_block.getByPosition(row_n);
|
|
||||||
columns[row_n]->insertFrom(*(col.column), 0);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
const auto & size = field.size;
|
|
||||||
columns[row_n]->insertData(static_cast<const char *>(field.data), size);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return sample_block.cloneWithColumns(std::move(columns));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
LibraryDictionarySource::LibraryDictionarySource(
|
LibraryDictionarySource::LibraryDictionarySource(
|
||||||
const DictionaryStructure & dict_struct_,
|
const DictionaryStructure & dict_struct_,
|
||||||
const Poco::Util::AbstractConfiguration & config,
|
const Poco::Util::AbstractConfiguration & config,
|
||||||
const std::string & config_prefix_,
|
const std::string & config_prefix_,
|
||||||
Block & sample_block_,
|
Block & sample_block_,
|
||||||
const Context & context,
|
const Context & context_,
|
||||||
bool check_config)
|
bool check_config)
|
||||||
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
||||||
, dict_struct{dict_struct_}
|
, dict_struct{dict_struct_}
|
||||||
, config_prefix{config_prefix_}
|
, config_prefix{config_prefix_}
|
||||||
, path{config.getString(config_prefix + ".path", "")}
|
, path{config.getString(config_prefix + ".path", "")}
|
||||||
|
, dictionary_id(getDictID())
|
||||||
, sample_block{sample_block_}
|
, sample_block{sample_block_}
|
||||||
|
, context(context_)
|
||||||
{
|
{
|
||||||
|
|
||||||
if (check_config)
|
if (check_config)
|
||||||
{
|
{
|
||||||
const String dictionaries_lib_path = context.getDictionariesLibPath();
|
const String dictionaries_lib_path = context.getDictionariesLibPath();
|
||||||
if (!startsWith(path, dictionaries_lib_path))
|
if (!startsWith(path, dictionaries_lib_path))
|
||||||
throw Exception("LibraryDictionarySource: Library path " + path + " is not inside " + dictionaries_lib_path, ErrorCodes::PATH_ACCESS_DENIED);
|
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!Poco::File(path).exists())
|
if (!Poco::File(path).exists())
|
||||||
throw Exception(
|
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path());
|
||||||
"LibraryDictionarySource: Can't load library " + Poco::File(path).path() + ": file doesn't exist",
|
|
||||||
ErrorCodes::FILE_DOESNT_EXIST);
|
|
||||||
|
|
||||||
description.init(sample_block);
|
description.init(sample_block);
|
||||||
library = std::make_shared<SharedLibrary>(path, RTLD_LAZY
|
bridge_helper = std::make_shared<LibraryBridgeHelper>(context, description.sample_block, dictionary_id);
|
||||||
#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
|
auto res = bridge_helper->initLibrary(path, getLibrarySettingsString(config, config_prefix + ".settings"), getDictAttributesString());
|
||||||
| RTLD_DEEPBIND
|
|
||||||
#endif
|
|
||||||
);
|
|
||||||
settings = std::make_shared<CStringsHolder>(getLibSettings(config, config_prefix + lib_config_settings));
|
|
||||||
|
|
||||||
if (auto lib_new = library->tryGet<ClickHouseLibrary::LibraryNewFunc>(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME))
|
if (!res)
|
||||||
lib_data = lib_new(&settings->strings, ClickHouseLibrary::log);
|
throw Exception(ErrorCodes::EXTERNAL_LIBRARY_ERROR, "Failed to create shared library from path: {}", path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
LibraryDictionarySource::~LibraryDictionarySource()
|
||||||
|
{
|
||||||
|
bridge_helper->removeLibrary();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
|
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
|
||||||
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
||||||
, dict_struct{other.dict_struct}
|
, dict_struct{other.dict_struct}
|
||||||
, config_prefix{other.config_prefix}
|
, config_prefix{other.config_prefix}
|
||||||
, path{other.path}
|
, path{other.path}
|
||||||
|
, dictionary_id{getDictID()}
|
||||||
, sample_block{other.sample_block}
|
, sample_block{other.sample_block}
|
||||||
, library{other.library}
|
, context(other.context)
|
||||||
, description{other.description}
|
, description{other.description}
|
||||||
, settings{other.settings}
|
|
||||||
{
|
{
|
||||||
if (auto lib_clone = library->tryGet<ClickHouseLibrary::LibraryCloneFunc>(ClickHouseLibrary::LIBRARY_CLONE_FUNC_NAME))
|
bridge_helper = std::make_shared<LibraryBridgeHelper>(context, description.sample_block, dictionary_id);
|
||||||
lib_data = lib_clone(other.lib_data);
|
bridge_helper->cloneLibrary(other.dictionary_id);
|
||||||
else if (auto lib_new = library->tryGet<ClickHouseLibrary::LibraryNewFunc>(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME))
|
|
||||||
lib_data = lib_new(&settings->strings, ClickHouseLibrary::log);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
LibraryDictionarySource::~LibraryDictionarySource()
|
|
||||||
|
bool LibraryDictionarySource::isModified() const
|
||||||
{
|
{
|
||||||
if (auto lib_delete = library->tryGet<ClickHouseLibrary::LibraryDeleteFunc>(ClickHouseLibrary::LIBRARY_DELETE_FUNC_NAME))
|
return bridge_helper->isModified();
|
||||||
lib_delete(lib_data);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool LibraryDictionarySource::supportsSelectiveLoad() const
|
||||||
|
{
|
||||||
|
return bridge_helper->supportsSelectiveLoad();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
BlockInputStreamPtr LibraryDictionarySource::loadAll()
|
BlockInputStreamPtr LibraryDictionarySource::loadAll()
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "loadAll {}", toString());
|
LOG_TRACE(log, "loadAll {}", toString());
|
||||||
|
return bridge_helper->loadAll();
|
||||||
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
|
|
||||||
ClickHouseLibrary::CStrings columns{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()),
|
|
||||||
dict_struct.attributes.size()};
|
|
||||||
size_t i = 0;
|
|
||||||
for (const auto & a : dict_struct.attributes)
|
|
||||||
{
|
|
||||||
columns.data[i] = a.name.c_str();
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
auto load_all_func = library->get<ClickHouseLibrary::LibraryLoadAllFunc>(ClickHouseLibrary::LIBRARY_LOAD_ALL_FUNC_NAME);
|
|
||||||
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
|
|
||||||
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME);
|
|
||||||
|
|
||||||
ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data);
|
|
||||||
SCOPE_EXIT(data_delete_func(lib_data, data_ptr));
|
|
||||||
|
|
||||||
ClickHouseLibrary::RawClickHouseLibraryTable data = load_all_func(data_ptr, &settings->strings, &columns);
|
|
||||||
auto block = dataToBlock(description.sample_block, data);
|
|
||||||
|
|
||||||
return std::make_shared<OneBlockInputStream>(block);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
||||||
|
return bridge_helper->loadIds(getDictIdsString(ids));
|
||||||
const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast<decltype(ClickHouseLibrary::VectorUInt64::data)>(ids.data()), ids.size()};
|
|
||||||
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
|
|
||||||
ClickHouseLibrary::CStrings columns_pass{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()),
|
|
||||||
dict_struct.attributes.size()};
|
|
||||||
size_t i = 0;
|
|
||||||
for (const auto & a : dict_struct.attributes)
|
|
||||||
{
|
|
||||||
columns_pass.data[i] = a.name.c_str();
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
auto load_ids_func = library->get<ClickHouseLibrary::LibraryLoadIdsFunc>(ClickHouseLibrary::LIBRARY_LOAD_IDS_FUNC_NAME);
|
|
||||||
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
|
|
||||||
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME);
|
|
||||||
|
|
||||||
ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data);
|
|
||||||
SCOPE_EXIT(data_delete_func(lib_data, data_ptr));
|
|
||||||
|
|
||||||
ClickHouseLibrary::RawClickHouseLibraryTable data = load_ids_func(data_ptr, &settings->strings, &columns_pass, &ids_data);
|
|
||||||
auto block = dataToBlock(description.sample_block, data);
|
|
||||||
|
|
||||||
return std::make_shared<OneBlockInputStream>(block);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
|
BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
||||||
|
auto block = blockForKeys(dict_struct, key_columns, requested_rows);
|
||||||
auto holder = std::make_unique<ClickHouseLibrary::Row[]>(key_columns.size());
|
return bridge_helper->loadKeys(block);
|
||||||
std::vector<std::unique_ptr<ClickHouseLibrary::Field[]>> column_data_holders;
|
|
||||||
for (size_t i = 0; i < key_columns.size(); ++i)
|
|
||||||
{
|
|
||||||
auto cell_holder = std::make_unique<ClickHouseLibrary::Field[]>(requested_rows.size());
|
|
||||||
for (size_t j = 0; j < requested_rows.size(); ++j)
|
|
||||||
{
|
|
||||||
auto data_ref = key_columns[i]->getDataAt(requested_rows[j]);
|
|
||||||
cell_holder[j] = ClickHouseLibrary::Field{.data = static_cast<const void *>(data_ref.data), .size = data_ref.size};
|
|
||||||
}
|
|
||||||
holder[i]
|
|
||||||
= ClickHouseLibrary::Row{.data = static_cast<ClickHouseLibrary::Field *>(cell_holder.get()), .size = requested_rows.size()};
|
|
||||||
|
|
||||||
column_data_holders.push_back(std::move(cell_holder));
|
|
||||||
}
|
|
||||||
|
|
||||||
ClickHouseLibrary::Table request_cols{.data = static_cast<ClickHouseLibrary::Row *>(holder.get()), .size = key_columns.size()};
|
|
||||||
|
|
||||||
auto load_keys_func = library->get<ClickHouseLibrary::LibraryLoadKeysFunc>(ClickHouseLibrary::LIBRARY_LOAD_KEYS_FUNC_NAME);
|
|
||||||
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
|
|
||||||
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME);
|
|
||||||
|
|
||||||
ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data);
|
|
||||||
SCOPE_EXIT(data_delete_func(lib_data, data_ptr));
|
|
||||||
|
|
||||||
ClickHouseLibrary::RawClickHouseLibraryTable data = load_keys_func(data_ptr, &settings->strings, &request_cols);
|
|
||||||
auto block = dataToBlock(description.sample_block, data);
|
|
||||||
|
|
||||||
return std::make_shared<OneBlockInputStream>(block);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
bool LibraryDictionarySource::isModified() const
|
|
||||||
{
|
|
||||||
if (auto func_is_modified = library->tryGet<ClickHouseLibrary::LibraryIsModifiedFunc>(
|
|
||||||
ClickHouseLibrary::LIBRARY_IS_MODIFIED_FUNC_NAME))
|
|
||||||
return func_is_modified(lib_data, &settings->strings);
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool LibraryDictionarySource::supportsSelectiveLoad() const
|
|
||||||
{
|
|
||||||
if (auto func_supports_selective_load = library->tryGet<ClickHouseLibrary::LibrarySupportsSelectiveLoadFunc>(
|
|
||||||
ClickHouseLibrary::LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME))
|
|
||||||
return func_supports_selective_load(lib_data, &settings->strings);
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
DictionarySourcePtr LibraryDictionarySource::clone() const
|
DictionarySourcePtr LibraryDictionarySource::clone() const
|
||||||
{
|
{
|
||||||
return std::make_unique<LibraryDictionarySource>(*this);
|
return std::make_unique<LibraryDictionarySource>(*this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
std::string LibraryDictionarySource::toString() const
|
std::string LibraryDictionarySource::toString() const
|
||||||
{
|
{
|
||||||
return path;
|
return path;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
String LibraryDictionarySource::getLibrarySettingsString(const Poco::Util::AbstractConfiguration & config, const std::string & config_root)
|
||||||
|
{
|
||||||
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||||
|
config.keys(config_root, config_keys);
|
||||||
|
WriteBufferFromOwnString out;
|
||||||
|
std::vector<std::string> settings;
|
||||||
|
|
||||||
|
for (const auto & key : config_keys)
|
||||||
|
{
|
||||||
|
std::string key_name = key;
|
||||||
|
auto bracket_pos = key.find('[');
|
||||||
|
|
||||||
|
if (bracket_pos != std::string::npos && bracket_pos > 0)
|
||||||
|
key_name = key.substr(0, bracket_pos);
|
||||||
|
|
||||||
|
settings.push_back(key_name);
|
||||||
|
settings.push_back(config.getString(config_root + "." + key));
|
||||||
|
}
|
||||||
|
|
||||||
|
writeVectorBinary(settings, out);
|
||||||
|
return out.str();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
String LibraryDictionarySource::getDictIdsString(const std::vector<UInt64> & ids)
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString out;
|
||||||
|
writeVectorBinary(ids, out);
|
||||||
|
return out.str();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
String LibraryDictionarySource::getDictAttributesString()
|
||||||
|
{
|
||||||
|
std::vector<String> attributes_names(dict_struct.attributes.size());
|
||||||
|
for (size_t i = 0; i < dict_struct.attributes.size(); ++i)
|
||||||
|
attributes_names[i] = dict_struct.attributes[i].name;
|
||||||
|
WriteBufferFromOwnString out;
|
||||||
|
writeVectorBinary(attributes_names, out);
|
||||||
|
return out.str();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
||||||
{
|
{
|
||||||
auto create_table_source = [=](const DictionaryStructure & dict_struct,
|
auto create_table_source = [=](const DictionaryStructure & dict_struct,
|
||||||
@ -306,7 +184,9 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
|||||||
{
|
{
|
||||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
|
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
|
||||||
};
|
};
|
||||||
|
|
||||||
factory.registerSource("library", create_table_source);
|
factory.registerSource("library", create_table_source);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,7 +1,9 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Common/SharedLibrary.h>
|
#include <Common/SharedLibrary.h>
|
||||||
|
#include <Bridge/LibraryBridgeHelper.h>
|
||||||
#include <common/LocalDateTime.h>
|
#include <common/LocalDateTime.h>
|
||||||
|
#include <Core/UUID.h>
|
||||||
#include "DictionaryStructure.h"
|
#include "DictionaryStructure.h"
|
||||||
#include <Core/ExternalResultDescription.h>
|
#include <Core/ExternalResultDescription.h>
|
||||||
#include "IDictionarySource.h"
|
#include "IDictionarySource.h"
|
||||||
@ -17,18 +19,17 @@ namespace Util
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NOT_IMPLEMENTED;
|
extern const int NOT_IMPLEMENTED;
|
||||||
}
|
}
|
||||||
class CStringsHolder;
|
|
||||||
|
|
||||||
/// Allows loading dictionaries from dynamic libraries (.so)
|
class CStringsHolder;
|
||||||
/// Experimental version
|
using LibraryBridgeHelperPtr = std::shared_ptr<LibraryBridgeHelper>;
|
||||||
/// Example: tests/external_dictionaries/dictionary_library/dictionary_library.cpp
|
|
||||||
class LibraryDictionarySource final : public IDictionarySource
|
class LibraryDictionarySource final : public IDictionarySource
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -37,7 +38,7 @@ public:
|
|||||||
const Poco::Util::AbstractConfiguration & config,
|
const Poco::Util::AbstractConfiguration & config,
|
||||||
const std::string & config_prefix_,
|
const std::string & config_prefix_,
|
||||||
Block & sample_block_,
|
Block & sample_block_,
|
||||||
const Context & context,
|
const Context & context_,
|
||||||
bool check_config);
|
bool check_config);
|
||||||
|
|
||||||
LibraryDictionarySource(const LibraryDictionarySource & other);
|
LibraryDictionarySource(const LibraryDictionarySource & other);
|
||||||
@ -68,18 +69,26 @@ public:
|
|||||||
std::string toString() const override;
|
std::string toString() const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Poco::Logger * log;
|
static String getDictIdsString(const std::vector<UInt64> & ids);
|
||||||
|
|
||||||
LocalDateTime getLastModification() const;
|
String getDictAttributesString();
|
||||||
|
|
||||||
|
static String getLibrarySettingsString(const Poco::Util::AbstractConfiguration & config, const std::string & config_root);
|
||||||
|
|
||||||
|
static Field getDictID() { return UUIDHelpers::generateV4(); }
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
|
||||||
const DictionaryStructure dict_struct;
|
const DictionaryStructure dict_struct;
|
||||||
const std::string config_prefix;
|
const std::string config_prefix;
|
||||||
const std::string path;
|
const std::string path;
|
||||||
|
const Field dictionary_id;
|
||||||
|
|
||||||
Block sample_block;
|
Block sample_block;
|
||||||
SharedLibraryPtr library;
|
Context context;
|
||||||
|
|
||||||
|
LibraryBridgeHelperPtr bridge_helper;
|
||||||
ExternalResultDescription description;
|
ExternalResultDescription description;
|
||||||
std::shared_ptr<CStringsHolder> settings;
|
|
||||||
void * lib_data = nullptr;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -70,7 +70,7 @@ MySQLDictionarySource::MySQLDictionarySource(
|
|||||||
, update_field{config.getString(config_prefix + ".update_field", "")}
|
, update_field{config.getString(config_prefix + ".update_field", "")}
|
||||||
, dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}
|
, dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}
|
||||||
, sample_block{sample_block_}
|
, sample_block{sample_block_}
|
||||||
, pool{mysqlxx::PoolFactory::instance().get(config, config_prefix)}
|
, pool{std::make_shared<mysqlxx::PoolWithFailover>(mysqlxx::PoolFactory::instance().get(config, config_prefix))}
|
||||||
, query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks}
|
, query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks}
|
||||||
, load_all_query{query_builder.composeLoadAllQuery()}
|
, load_all_query{query_builder.composeLoadAllQuery()}
|
||||||
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
|
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
|
||||||
@ -119,67 +119,43 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr MySQLDictionarySource::retriedCreateMySqlBIStream(const std::string & data_fetch_query_str, const size_t max_tries)
|
BlockInputStreamPtr MySQLDictionarySource::loadFromQuery(const String & query)
|
||||||
{
|
{
|
||||||
size_t count_connection_lost = 0;
|
return std::make_shared<MySQLWithFailoverBlockInputStream>(
|
||||||
|
pool, query, sample_block, max_block_size, close_connection, false, max_tries_for_mysql_block_input_stream);
|
||||||
while (true)
|
|
||||||
{
|
|
||||||
auto connection = pool.get();
|
|
||||||
|
|
||||||
try
|
|
||||||
{
|
|
||||||
return std::make_shared<MySQLBlockInputStream>(
|
|
||||||
connection, data_fetch_query_str, sample_block, max_block_size, close_connection);
|
|
||||||
}
|
|
||||||
catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST
|
|
||||||
{
|
|
||||||
if (++count_connection_lost < max_tries)
|
|
||||||
{
|
|
||||||
LOG_WARNING(log, ecl.displayText());
|
|
||||||
LOG_WARNING(log, "Lost connection ({}/{}). Trying to reconnect...", count_connection_lost, max_tries);
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG_ERROR(log, "Failed ({}/{}) to create BlockInputStream for MySQL dictionary source.", count_connection_lost, max_tries);
|
|
||||||
throw;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr MySQLDictionarySource::loadAll()
|
BlockInputStreamPtr MySQLDictionarySource::loadAll()
|
||||||
{
|
{
|
||||||
auto connection = pool.get();
|
auto connection = pool->get();
|
||||||
last_modification = getLastModification(connection, false);
|
last_modification = getLastModification(connection, false);
|
||||||
|
|
||||||
LOG_TRACE(log, load_all_query);
|
LOG_TRACE(log, load_all_query);
|
||||||
return retriedCreateMySqlBIStream(load_all_query, max_tries_for_mysql_block_input_stream);
|
return loadFromQuery(load_all_query);
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
|
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
|
||||||
{
|
{
|
||||||
auto connection = pool.get();
|
auto connection = pool->get();
|
||||||
last_modification = getLastModification(connection, false);
|
last_modification = getLastModification(connection, false);
|
||||||
|
|
||||||
std::string load_update_query = getUpdateFieldAndDate();
|
std::string load_update_query = getUpdateFieldAndDate();
|
||||||
LOG_TRACE(log, load_update_query);
|
LOG_TRACE(log, load_update_query);
|
||||||
return retriedCreateMySqlBIStream(load_update_query, max_tries_for_mysql_block_input_stream);
|
return loadFromQuery(load_update_query);
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||||
{
|
{
|
||||||
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
|
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
|
||||||
|
|
||||||
const auto query = query_builder.composeLoadIdsQuery(ids);
|
const auto query = query_builder.composeLoadIdsQuery(ids);
|
||||||
return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream);
|
return loadFromQuery(query);
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
|
BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
|
||||||
{
|
{
|
||||||
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
|
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
|
||||||
|
|
||||||
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
|
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
|
||||||
return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream);
|
return loadFromQuery(query);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MySQLDictionarySource::isModified() const
|
bool MySQLDictionarySource::isModified() const
|
||||||
@ -195,7 +171,7 @@ bool MySQLDictionarySource::isModified() const
|
|||||||
|
|
||||||
if (dont_check_update_time)
|
if (dont_check_update_time)
|
||||||
return true;
|
return true;
|
||||||
auto connection = pool.get();
|
auto connection = pool->get();
|
||||||
return getLastModification(connection, true) > last_modification;
|
return getLastModification(connection, true) > last_modification;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -293,7 +269,7 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request
|
|||||||
Block invalidate_sample_block;
|
Block invalidate_sample_block;
|
||||||
ColumnPtr column(ColumnString::create());
|
ColumnPtr column(ColumnString::create());
|
||||||
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
||||||
MySQLBlockInputStream block_input_stream(pool.get(), request, invalidate_sample_block, 1, close_connection);
|
MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, 1, close_connection);
|
||||||
return readInvalidateQuery(block_input_stream);
|
return readInvalidateQuery(block_input_stream);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -34,7 +34,7 @@ public:
|
|||||||
MySQLDictionarySource(
|
MySQLDictionarySource(
|
||||||
const DictionaryStructure & dict_struct_,
|
const DictionaryStructure & dict_struct_,
|
||||||
const Poco::Util::AbstractConfiguration & config,
|
const Poco::Util::AbstractConfiguration & config,
|
||||||
const std::string & config_prefix,
|
const String & config_prefix,
|
||||||
const Block & sample_block_);
|
const Block & sample_block_);
|
||||||
|
|
||||||
/// copy-constructor is provided in order to support cloneability
|
/// copy-constructor is provided in order to support cloneability
|
||||||
@ -60,6 +60,8 @@ public:
|
|||||||
std::string toString() const override;
|
std::string toString() const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
BlockInputStreamPtr loadFromQuery(const String & query);
|
||||||
|
|
||||||
std::string getUpdateFieldAndDate();
|
std::string getUpdateFieldAndDate();
|
||||||
|
|
||||||
static std::string quoteForLike(const std::string s);
|
static std::string quoteForLike(const std::string s);
|
||||||
@ -69,9 +71,6 @@ private:
|
|||||||
// execute invalidate_query. expects single cell in result
|
// execute invalidate_query. expects single cell in result
|
||||||
std::string doInvalidateQuery(const std::string & request) const;
|
std::string doInvalidateQuery(const std::string & request) const;
|
||||||
|
|
||||||
/// A helper method for recovering from "Lost connection to MySQL server during query" errors
|
|
||||||
BlockInputStreamPtr retriedCreateMySqlBIStream(const std::string & query_str, const size_t max_tries);
|
|
||||||
|
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
std::chrono::time_point<std::chrono::system_clock> update_time;
|
std::chrono::time_point<std::chrono::system_clock> update_time;
|
||||||
@ -82,7 +81,7 @@ private:
|
|||||||
const std::string update_field;
|
const std::string update_field;
|
||||||
const bool dont_check_update_time;
|
const bool dont_check_update_time;
|
||||||
Block sample_block;
|
Block sample_block;
|
||||||
mutable mysqlxx::PoolWithFailover pool;
|
mutable mysqlxx::PoolWithFailoverPtr pool;
|
||||||
ExternalQueryBuilder query_builder;
|
ExternalQueryBuilder query_builder;
|
||||||
const std::string load_all_query;
|
const std::string load_all_query;
|
||||||
LocalDateTime last_modification;
|
LocalDateTime last_modification;
|
||||||
|
@ -31,7 +31,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(
|
|||||||
const Block & sample_block_)
|
const Block & sample_block_)
|
||||||
: dict_struct{dict_struct_}
|
: dict_struct{dict_struct_}
|
||||||
, sample_block(sample_block_)
|
, sample_block(sample_block_)
|
||||||
, connection(std::make_shared<PostgreSQLReplicaConnection>(config_, config_prefix))
|
, connection(std::make_shared<postgres::PoolWithFailover>(config_, config_prefix))
|
||||||
, log(&Poco::Logger::get("PostgreSQLDictionarySource"))
|
, log(&Poco::Logger::get("PostgreSQLDictionarySource"))
|
||||||
, db(config_.getString(fmt::format("{}.db", config_prefix), ""))
|
, db(config_.getString(fmt::format("{}.db", config_prefix), ""))
|
||||||
, table(config_.getString(fmt::format("{}.table", config_prefix), ""))
|
, table(config_.getString(fmt::format("{}.table", config_prefix), ""))
|
||||||
|
@ -11,7 +11,7 @@
|
|||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
#include <common/LocalDateTime.h>
|
#include <common/LocalDateTime.h>
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include <Storages/PostgreSQL/PostgreSQLReplicaConnection.h>
|
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||||
#include <pqxx/pqxx>
|
#include <pqxx/pqxx>
|
||||||
|
|
||||||
|
|
||||||
@ -51,7 +51,7 @@ private:
|
|||||||
|
|
||||||
const DictionaryStructure dict_struct;
|
const DictionaryStructure dict_struct;
|
||||||
Block sample_block;
|
Block sample_block;
|
||||||
PostgreSQLReplicaConnectionPtr connection;
|
postgres::PoolWithFailoverPtr connection;
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
const std::string db;
|
const std::string db;
|
||||||
|
@ -11,7 +11,6 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Poco/Net/HTTPRequest.h>
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
#include <Common/XDBCBridgeHelper.h>
|
|
||||||
#include <common/LocalDateTime.h>
|
#include <common/LocalDateTime.h>
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include "DictionarySourceFactory.h"
|
#include "DictionarySourceFactory.h"
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
#include <IO/ConnectionTimeouts.h>
|
#include <IO/ConnectionTimeouts.h>
|
||||||
#include <Poco/Data/SessionPool.h>
|
#include <Poco/Data/SessionPool.h>
|
||||||
#include <Poco/URI.h>
|
#include <Poco/URI.h>
|
||||||
#include <Common/XDBCBridgeHelper.h>
|
#include <Bridge/XDBCBridgeHelper.h>
|
||||||
#include "DictionaryStructure.h"
|
#include "DictionaryStructure.h"
|
||||||
#include "ExternalQueryBuilder.h"
|
#include "ExternalQueryBuilder.h"
|
||||||
#include "IDictionarySource.h"
|
#include "IDictionarySource.h"
|
||||||
|
@ -1,26 +1,29 @@
|
|||||||
#if !defined(ARCADIA_BUILD)
|
#if !defined(ARCADIA_BUILD)
|
||||||
# include "config_core.h"
|
#include "config_core.h"
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
# include <vector>
|
#include <vector>
|
||||||
# include <Columns/ColumnNullable.h>
|
#include <Columns/ColumnNullable.h>
|
||||||
# include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
# include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
# include <Columns/ColumnDecimal.h>
|
#include <Columns/ColumnDecimal.h>
|
||||||
# include <Columns/ColumnFixedString.h>
|
#include <Columns/ColumnFixedString.h>
|
||||||
# include <DataTypes/IDataType.h>
|
#include <DataTypes/IDataType.h>
|
||||||
# include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
# include <IO/ReadBufferFromString.h>
|
#include <IO/ReadBufferFromString.h>
|
||||||
# include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
# include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
# include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
# include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
# include <ext/range.h>
|
#include <ext/range.h>
|
||||||
# include "MySQLBlockInputStream.h"
|
#include <common/logger_useful.h>
|
||||||
|
#include "MySQLBlockInputStream.h"
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
||||||
@ -36,6 +39,7 @@ MySQLBlockInputStream::Connection::Connection(
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Used in MaterializeMySQL and in doInvalidateQuery for dictionary source.
|
||||||
MySQLBlockInputStream::MySQLBlockInputStream(
|
MySQLBlockInputStream::MySQLBlockInputStream(
|
||||||
const mysqlxx::PoolWithFailover::Entry & entry,
|
const mysqlxx::PoolWithFailover::Entry & entry,
|
||||||
const std::string & query_str,
|
const std::string & query_str,
|
||||||
@ -43,7 +47,8 @@ MySQLBlockInputStream::MySQLBlockInputStream(
|
|||||||
const UInt64 max_block_size_,
|
const UInt64 max_block_size_,
|
||||||
const bool auto_close_,
|
const bool auto_close_,
|
||||||
const bool fetch_by_name_)
|
const bool fetch_by_name_)
|
||||||
: connection{std::make_unique<Connection>(entry, query_str)}
|
: log(&Poco::Logger::get("MySQLBlockInputStream"))
|
||||||
|
, connection{std::make_unique<Connection>(entry, query_str)}
|
||||||
, max_block_size{max_block_size_}
|
, max_block_size{max_block_size_}
|
||||||
, auto_close{auto_close_}
|
, auto_close{auto_close_}
|
||||||
, fetch_by_name(fetch_by_name_)
|
, fetch_by_name(fetch_by_name_)
|
||||||
@ -52,6 +57,62 @@ MySQLBlockInputStream::MySQLBlockInputStream(
|
|||||||
initPositionMappingFromQueryResultStructure();
|
initPositionMappingFromQueryResultStructure();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// For descendant MySQLWithFailoverBlockInputStream
|
||||||
|
MySQLBlockInputStream::MySQLBlockInputStream(
|
||||||
|
const Block & sample_block_,
|
||||||
|
UInt64 max_block_size_,
|
||||||
|
bool auto_close_,
|
||||||
|
bool fetch_by_name_)
|
||||||
|
: log(&Poco::Logger::get("MySQLBlockInputStream"))
|
||||||
|
, max_block_size(max_block_size_)
|
||||||
|
, auto_close(auto_close_)
|
||||||
|
, fetch_by_name(fetch_by_name_)
|
||||||
|
{
|
||||||
|
description.init(sample_block_);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Used by MySQL storage / table function and dictionary source.
|
||||||
|
MySQLWithFailoverBlockInputStream::MySQLWithFailoverBlockInputStream(
|
||||||
|
mysqlxx::PoolWithFailoverPtr pool_,
|
||||||
|
const std::string & query_str_,
|
||||||
|
const Block & sample_block_,
|
||||||
|
const UInt64 max_block_size_,
|
||||||
|
const bool auto_close_,
|
||||||
|
const bool fetch_by_name_,
|
||||||
|
const size_t max_tries_)
|
||||||
|
: MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_, fetch_by_name_)
|
||||||
|
, pool(pool_)
|
||||||
|
, query_str(query_str_)
|
||||||
|
, max_tries(max_tries_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
void MySQLWithFailoverBlockInputStream::readPrefix()
|
||||||
|
{
|
||||||
|
size_t count_connect_attempts = 0;
|
||||||
|
|
||||||
|
/// For recovering from "Lost connection to MySQL server during query" errors
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
connection = std::make_unique<Connection>(pool->get(), query_str);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST
|
||||||
|
{
|
||||||
|
LOG_WARNING(log, "Failed connection ({}/{}). Trying to reconnect... (Info: {})", count_connect_attempts, max_tries, ecl.displayText());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (++count_connect_attempts > max_tries)
|
||||||
|
{
|
||||||
|
LOG_ERROR(log, "Failed to create connection to MySQL. ({}/{})", count_connect_attempts, max_tries);
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
initPositionMappingFromQueryResultStructure();
|
||||||
|
}
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
@ -139,6 +200,7 @@ Block MySQLBlockInputStream::readImpl()
|
|||||||
{
|
{
|
||||||
if (auto_close)
|
if (auto_close)
|
||||||
connection->entry.disconnect();
|
connection->entry.disconnect();
|
||||||
|
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -191,18 +253,6 @@ Block MySQLBlockInputStream::readImpl()
|
|||||||
return description.sample_block.cloneWithColumns(std::move(columns));
|
return description.sample_block.cloneWithColumns(std::move(columns));
|
||||||
}
|
}
|
||||||
|
|
||||||
MySQLBlockInputStream::MySQLBlockInputStream(
|
|
||||||
const Block & sample_block_,
|
|
||||||
UInt64 max_block_size_,
|
|
||||||
bool auto_close_,
|
|
||||||
bool fetch_by_name_)
|
|
||||||
: max_block_size(max_block_size_)
|
|
||||||
, auto_close(auto_close_)
|
|
||||||
, fetch_by_name(fetch_by_name_)
|
|
||||||
{
|
|
||||||
description.init(sample_block_);
|
|
||||||
}
|
|
||||||
|
|
||||||
void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure()
|
void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure()
|
||||||
{
|
{
|
||||||
position_mapping.resize(description.sample_block.columns());
|
position_mapping.resize(description.sample_block.columns());
|
||||||
@ -250,25 +300,6 @@ void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
MySQLLazyBlockInputStream::MySQLLazyBlockInputStream(
|
|
||||||
mysqlxx::Pool & pool_,
|
|
||||||
const std::string & query_str_,
|
|
||||||
const Block & sample_block_,
|
|
||||||
const UInt64 max_block_size_,
|
|
||||||
const bool auto_close_,
|
|
||||||
const bool fetch_by_name_)
|
|
||||||
: MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_, fetch_by_name_)
|
|
||||||
, pool(pool_)
|
|
||||||
, query_str(query_str_)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
void MySQLLazyBlockInputStream::readPrefix()
|
|
||||||
{
|
|
||||||
connection = std::make_unique<Connection>(pool.get(), query_str);
|
|
||||||
initPositionMappingFromQueryResultStructure();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -41,6 +41,7 @@ protected:
|
|||||||
mysqlxx::UseQueryResult result;
|
mysqlxx::UseQueryResult result;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
std::unique_ptr<Connection> connection;
|
std::unique_ptr<Connection> connection;
|
||||||
|
|
||||||
const UInt64 max_block_size;
|
const UInt64 max_block_size;
|
||||||
@ -52,22 +53,27 @@ protected:
|
|||||||
|
|
||||||
/// Like MySQLBlockInputStream, but allocates connection only when reading is starting.
|
/// Like MySQLBlockInputStream, but allocates connection only when reading is starting.
|
||||||
/// It allows to create a lot of stream objects without occupation of all connection pool.
|
/// It allows to create a lot of stream objects without occupation of all connection pool.
|
||||||
class MySQLLazyBlockInputStream final : public MySQLBlockInputStream
|
/// Also makes attempts to reconnect in case of connection failures.
|
||||||
|
class MySQLWithFailoverBlockInputStream final : public MySQLBlockInputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
MySQLLazyBlockInputStream(
|
static constexpr inline auto MAX_TRIES_MYSQL_CONNECT = 5;
|
||||||
mysqlxx::Pool & pool_,
|
|
||||||
|
MySQLWithFailoverBlockInputStream(
|
||||||
|
mysqlxx::PoolWithFailoverPtr pool_,
|
||||||
const std::string & query_str_,
|
const std::string & query_str_,
|
||||||
const Block & sample_block_,
|
const Block & sample_block_,
|
||||||
const UInt64 max_block_size_,
|
const UInt64 max_block_size_,
|
||||||
const bool auto_close_ = false,
|
const bool auto_close_ = false,
|
||||||
const bool fetch_by_name_ = false);
|
const bool fetch_by_name_ = false,
|
||||||
|
const size_t max_tries_ = MAX_TRIES_MYSQL_CONNECT);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void readPrefix() override;
|
void readPrefix() override;
|
||||||
|
|
||||||
mysqlxx::Pool & pool;
|
mysqlxx::PoolWithFailoverPtr pool;
|
||||||
std::string query_str;
|
std::string query_str;
|
||||||
|
size_t max_tries;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2358,7 +2358,7 @@ void Context::setQueryParameter(const String & name, const String & value)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
|
void Context::addBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
|
||||||
{
|
{
|
||||||
auto lock = getLock();
|
auto lock = getLock();
|
||||||
shared->bridge_commands.emplace_back(std::move(cmd));
|
shared->bridge_commands.emplace_back(std::move(cmd));
|
||||||
|
@ -747,7 +747,7 @@ public:
|
|||||||
void setQueryParameters(const NameToNameMap & parameters) { query_parameters = parameters; }
|
void setQueryParameters(const NameToNameMap & parameters) { query_parameters = parameters; }
|
||||||
|
|
||||||
/// Add started bridge command. It will be killed after context destruction
|
/// Add started bridge command. It will be killed after context destruction
|
||||||
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
|
void addBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
|
||||||
|
|
||||||
IHostContextPtr & getHostContext();
|
IHostContextPtr & getHostContext();
|
||||||
const IHostContextPtr & getHostContext() const;
|
const IHostContextPtr & getHostContext() const;
|
||||||
|
@ -522,6 +522,10 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum
|
|||||||
|
|
||||||
for (const auto & column : columns)
|
for (const auto & column : columns)
|
||||||
{
|
{
|
||||||
|
/// Skip .sizeX and similar meta information
|
||||||
|
if (!column.getSubcolumnName().empty())
|
||||||
|
continue;
|
||||||
|
|
||||||
/// @todo resolve evil constant
|
/// @todo resolve evil constant
|
||||||
size_t size = column.type->haveMaximumSizeOfValue() ? column.type->getMaximumSizeOfValueInMemory() : 100;
|
size_t size = column.type->haveMaximumSizeOfValue() ? column.type->getMaximumSizeOfValueInMemory() : 100;
|
||||||
|
|
||||||
|
@ -9,21 +9,17 @@
|
|||||||
#include <DataStreams/NullAndDoCopyBlockInputStream.h>
|
#include <DataStreams/NullAndDoCopyBlockInputStream.h>
|
||||||
#include <DataStreams/NullBlockOutputStream.h>
|
#include <DataStreams/NullBlockOutputStream.h>
|
||||||
#include <DataStreams/PushingToViewsBlockOutputStream.h>
|
#include <DataStreams/PushingToViewsBlockOutputStream.h>
|
||||||
#include <DataStreams/RemoteBlockInputStream.h>
|
|
||||||
#include <DataStreams/SquashingBlockOutputStream.h>
|
#include <DataStreams/SquashingBlockOutputStream.h>
|
||||||
#include <DataStreams/copyData.h>
|
#include <DataStreams/copyData.h>
|
||||||
#include <IO/ConnectionTimeoutsContext.h>
|
#include <IO/ConnectionTimeoutsContext.h>
|
||||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||||
#include <Interpreters/InterpreterWatchQuery.h>
|
#include <Interpreters/InterpreterWatchQuery.h>
|
||||||
#include <Interpreters/JoinedTables.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTInsertQuery.h>
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Parsers/queryToString.h>
|
|
||||||
#include <Processors/NullSink.h>
|
|
||||||
#include <Processors/Sources/SinkToOutputStream.h>
|
#include <Processors/Sources/SinkToOutputStream.h>
|
||||||
#include <Processors/Sources/SourceFromInputStream.h>
|
#include <Processors/Sources/SourceFromInputStream.h>
|
||||||
#include <Processors/Transforms/ExpressionTransform.h>
|
#include <Processors/Transforms/ExpressionTransform.h>
|
||||||
@ -36,11 +32,6 @@
|
|||||||
#include <Interpreters/getTableExpressions.h>
|
#include <Interpreters/getTableExpressions.h>
|
||||||
#include <Interpreters/processColumnTransformers.h>
|
#include <Interpreters/processColumnTransformers.h>
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
const UInt64 PARALLEL_DISTRIBUTED_INSERT_SELECT_ALL = 2;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -50,7 +41,6 @@ namespace ErrorCodes
|
|||||||
extern const int NO_SUCH_COLUMN_IN_TABLE;
|
extern const int NO_SUCH_COLUMN_IN_TABLE;
|
||||||
extern const int ILLEGAL_COLUMN;
|
extern const int ILLEGAL_COLUMN;
|
||||||
extern const int DUPLICATE_COLUMN;
|
extern const int DUPLICATE_COLUMN;
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
InterpreterInsertQuery::InterpreterInsertQuery(
|
InterpreterInsertQuery::InterpreterInsertQuery(
|
||||||
@ -178,79 +168,10 @@ BlockIO InterpreterInsertQuery::execute()
|
|||||||
if (query.select && table->isRemote() && settings.parallel_distributed_insert_select)
|
if (query.select && table->isRemote() && settings.parallel_distributed_insert_select)
|
||||||
{
|
{
|
||||||
// Distributed INSERT SELECT
|
// Distributed INSERT SELECT
|
||||||
std::shared_ptr<StorageDistributed> storage_src;
|
if (auto maybe_pipeline = table->distributedWrite(query, context))
|
||||||
auto & select = query.select->as<ASTSelectWithUnionQuery &>();
|
|
||||||
auto new_query = std::dynamic_pointer_cast<ASTInsertQuery>(query.clone());
|
|
||||||
if (select.list_of_selects->children.size() == 1)
|
|
||||||
{
|
|
||||||
if (auto * select_query = select.list_of_selects->children.at(0)->as<ASTSelectQuery>())
|
|
||||||
{
|
|
||||||
JoinedTables joined_tables(Context(context), *select_query);
|
|
||||||
|
|
||||||
if (joined_tables.tablesCount() == 1)
|
|
||||||
{
|
|
||||||
storage_src = std::dynamic_pointer_cast<StorageDistributed>(joined_tables.getLeftTableStorage());
|
|
||||||
if (storage_src)
|
|
||||||
{
|
|
||||||
const auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
|
|
||||||
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
|
|
||||||
|
|
||||||
auto new_select_query = std::dynamic_pointer_cast<ASTSelectQuery>(select_query->clone());
|
|
||||||
select_with_union_query->list_of_selects->children.push_back(new_select_query);
|
|
||||||
|
|
||||||
new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName());
|
|
||||||
|
|
||||||
new_query->select = select_with_union_query;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
auto storage_dst = std::dynamic_pointer_cast<StorageDistributed>(table);
|
|
||||||
|
|
||||||
if (storage_src && storage_dst && storage_src->getClusterName() == storage_dst->getClusterName())
|
|
||||||
{
|
{
|
||||||
|
res.pipeline = std::move(*maybe_pipeline);
|
||||||
is_distributed_insert_select = true;
|
is_distributed_insert_select = true;
|
||||||
|
|
||||||
if (settings.parallel_distributed_insert_select == PARALLEL_DISTRIBUTED_INSERT_SELECT_ALL)
|
|
||||||
{
|
|
||||||
new_query->table_id = StorageID(storage_dst->getRemoteDatabaseName(), storage_dst->getRemoteTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
const auto & cluster = storage_src->getCluster();
|
|
||||||
const auto & shards_info = cluster->getShardsInfo();
|
|
||||||
|
|
||||||
std::vector<std::unique_ptr<QueryPipeline>> pipelines;
|
|
||||||
|
|
||||||
String new_query_str = queryToString(new_query);
|
|
||||||
for (size_t shard_index : ext::range(0, shards_info.size()))
|
|
||||||
{
|
|
||||||
const auto & shard_info = shards_info[shard_index];
|
|
||||||
if (shard_info.isLocal())
|
|
||||||
{
|
|
||||||
InterpreterInsertQuery interpreter(new_query, context);
|
|
||||||
pipelines.emplace_back(std::make_unique<QueryPipeline>(interpreter.execute().pipeline));
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
|
|
||||||
auto connections = shard_info.pool->getMany(timeouts, &settings, PoolMode::GET_ONE);
|
|
||||||
if (connections.empty() || connections.front().isNull())
|
|
||||||
throw Exception(
|
|
||||||
"Expected exactly one connection for shard " + toString(shard_info.shard_num), ErrorCodes::LOGICAL_ERROR);
|
|
||||||
|
|
||||||
/// INSERT SELECT query returns empty block
|
|
||||||
auto in_stream = std::make_shared<RemoteBlockInputStream>(std::move(connections), new_query_str, Block{}, context);
|
|
||||||
pipelines.emplace_back(std::make_unique<QueryPipeline>());
|
|
||||||
pipelines.back()->init(Pipe(std::make_shared<SourceFromInputStream>(std::move(in_stream))));
|
|
||||||
pipelines.back()->setSinks([](const Block & header, QueryPipeline::StreamType) -> ProcessorPtr
|
|
||||||
{
|
|
||||||
return std::make_shared<EmptySink>(header);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
res.pipeline = QueryPipeline::unitePipelines(std::move(pipelines), {}, ExpressionActionsSettings::fromContext(context));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -15,8 +15,16 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int UNEXPECTED_EXPRESSION;
|
||||||
|
}
|
||||||
|
|
||||||
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||||
{
|
{
|
||||||
|
if (name == "view")
|
||||||
|
throw Exception("Table function view cannot be used as an expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||||
|
|
||||||
writeString(name, ostr);
|
writeString(name, ostr);
|
||||||
|
|
||||||
if (parameters)
|
if (parameters)
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include <Storages/StorageInMemoryMetadata.h>
|
#include <Storages/StorageInMemoryMetadata.h>
|
||||||
#include <Storages/ColumnDependency.h>
|
#include <Storages/ColumnDependency.h>
|
||||||
#include <Storages/SelectQueryDescription.h>
|
#include <Storages/SelectQueryDescription.h>
|
||||||
|
#include <Processors/QueryPipeline.h>
|
||||||
#include <Common/ActionLock.h>
|
#include <Common/ActionLock.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/RWLock.h>
|
#include <Common/RWLock.h>
|
||||||
@ -34,6 +35,7 @@ class Context;
|
|||||||
using StorageActionBlockType = size_t;
|
using StorageActionBlockType = size_t;
|
||||||
|
|
||||||
class ASTCreateQuery;
|
class ASTCreateQuery;
|
||||||
|
class ASTInsertQuery;
|
||||||
|
|
||||||
struct Settings;
|
struct Settings;
|
||||||
|
|
||||||
@ -50,6 +52,9 @@ class Pipe;
|
|||||||
class QueryPlan;
|
class QueryPlan;
|
||||||
using QueryPlanPtr = std::unique_ptr<QueryPlan>;
|
using QueryPlanPtr = std::unique_ptr<QueryPlan>;
|
||||||
|
|
||||||
|
class QueryPipeline;
|
||||||
|
using QueryPipelinePtr = std::unique_ptr<QueryPipeline>;
|
||||||
|
|
||||||
class IStoragePolicy;
|
class IStoragePolicy;
|
||||||
using StoragePolicyPtr = std::shared_ptr<const IStoragePolicy>;
|
using StoragePolicyPtr = std::shared_ptr<const IStoragePolicy>;
|
||||||
|
|
||||||
@ -319,6 +324,19 @@ public:
|
|||||||
throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Writes the data to a table in distributed manner.
|
||||||
|
* It is supposed that implementation looks into SELECT part of the query and executes distributed
|
||||||
|
* INSERT SELECT if it is possible with current storage as a receiver and query SELECT part as a producer.
|
||||||
|
*
|
||||||
|
* Returns query pipeline if distributed writing is possible, and nullptr otherwise.
|
||||||
|
*/
|
||||||
|
virtual QueryPipelinePtr distributedWrite(
|
||||||
|
const ASTInsertQuery & /*query*/,
|
||||||
|
const Context & /*context*/)
|
||||||
|
{
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
/** Delete the table data. Called before deleting the directory with the data.
|
/** Delete the table data. Called before deleting the directory with the data.
|
||||||
* The method can be called only after detaching table from Context (when no queries are performed with table).
|
* The method can be called only after detaching table from Context (when no queries are performed with table).
|
||||||
* The table is not usable during and after call to this method.
|
* The table is not usable during and after call to this method.
|
||||||
|
@ -8,10 +8,10 @@
|
|||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace postgres
|
||||||
{
|
{
|
||||||
|
|
||||||
PostgreSQLConnection::PostgreSQLConnection(
|
Connection::Connection(
|
||||||
const String & connection_str_,
|
const String & connection_str_,
|
||||||
const String & address_)
|
const String & address_)
|
||||||
: connection_str(connection_str_)
|
: connection_str(connection_str_)
|
||||||
@ -20,14 +20,14 @@ PostgreSQLConnection::PostgreSQLConnection(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLConnection::ConnectionPtr PostgreSQLConnection::get()
|
pqxx::ConnectionPtr Connection::get()
|
||||||
{
|
{
|
||||||
connectIfNeeded();
|
connectIfNeeded();
|
||||||
return connection;
|
return connection;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLConnection::ConnectionPtr PostgreSQLConnection::tryGet()
|
pqxx::ConnectionPtr Connection::tryGet()
|
||||||
{
|
{
|
||||||
if (tryConnectIfNeeded())
|
if (tryConnectIfNeeded())
|
||||||
return connection;
|
return connection;
|
||||||
@ -35,7 +35,7 @@ PostgreSQLConnection::ConnectionPtr PostgreSQLConnection::tryGet()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void PostgreSQLConnection::connectIfNeeded()
|
void Connection::connectIfNeeded()
|
||||||
{
|
{
|
||||||
if (!connection || !connection->is_open())
|
if (!connection || !connection->is_open())
|
||||||
{
|
{
|
||||||
@ -45,7 +45,7 @@ void PostgreSQLConnection::connectIfNeeded()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool PostgreSQLConnection::tryConnectIfNeeded()
|
bool Connection::tryConnectIfNeeded()
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
@ -10,24 +10,27 @@
|
|||||||
#include <Common/ConcurrentBoundedQueue.h>
|
#include <Common/ConcurrentBoundedQueue.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace pqxx
|
||||||
|
{
|
||||||
|
using ConnectionPtr = std::shared_ptr<pqxx::connection>;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace postgres
|
||||||
{
|
{
|
||||||
|
|
||||||
class PostgreSQLConnection
|
class Connection
|
||||||
{
|
{
|
||||||
|
|
||||||
using ConnectionPtr = std::shared_ptr<pqxx::connection>;
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
PostgreSQLConnection(
|
Connection(
|
||||||
const String & connection_str_,
|
const String & connection_str_,
|
||||||
const String & address_);
|
const String & address_);
|
||||||
|
|
||||||
PostgreSQLConnection(const PostgreSQLConnection & other) = delete;
|
Connection(const Connection & other) = delete;
|
||||||
|
|
||||||
ConnectionPtr get();
|
pqxx::ConnectionPtr get();
|
||||||
|
|
||||||
ConnectionPtr tryGet();
|
pqxx::ConnectionPtr tryGet();
|
||||||
|
|
||||||
bool isConnected() { return tryConnectIfNeeded(); }
|
bool isConnected() { return tryConnectIfNeeded(); }
|
||||||
|
|
||||||
@ -38,40 +41,40 @@ private:
|
|||||||
|
|
||||||
const std::string & getAddress() { return address; }
|
const std::string & getAddress() { return address; }
|
||||||
|
|
||||||
ConnectionPtr connection;
|
pqxx::ConnectionPtr connection;
|
||||||
std::string connection_str, address;
|
std::string connection_str, address;
|
||||||
};
|
};
|
||||||
|
|
||||||
using PostgreSQLConnectionPtr = std::shared_ptr<PostgreSQLConnection>;
|
using ConnectionPtr = std::shared_ptr<Connection>;
|
||||||
|
|
||||||
|
|
||||||
class PostgreSQLConnectionHolder
|
class ConnectionHolder
|
||||||
{
|
{
|
||||||
|
|
||||||
using Pool = ConcurrentBoundedQueue<PostgreSQLConnectionPtr>;
|
using Pool = ConcurrentBoundedQueue<ConnectionPtr>;
|
||||||
static constexpr inline auto POSTGRESQL_POOL_WAIT_MS = 50;
|
static constexpr inline auto POSTGRESQL_POOL_WAIT_MS = 50;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
PostgreSQLConnectionHolder(PostgreSQLConnectionPtr connection_, Pool & pool_)
|
ConnectionHolder(ConnectionPtr connection_, Pool & pool_)
|
||||||
: connection(std::move(connection_))
|
: connection(std::move(connection_))
|
||||||
, pool(pool_)
|
, pool(pool_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
PostgreSQLConnectionHolder(const PostgreSQLConnectionHolder & other) = delete;
|
ConnectionHolder(const ConnectionHolder & other) = delete;
|
||||||
|
|
||||||
~PostgreSQLConnectionHolder() { pool.tryPush(connection, POSTGRESQL_POOL_WAIT_MS); }
|
~ConnectionHolder() { pool.tryPush(connection, POSTGRESQL_POOL_WAIT_MS); }
|
||||||
|
|
||||||
pqxx::connection & conn() const { return *connection->get(); }
|
pqxx::connection & conn() const { return *connection->get(); }
|
||||||
|
|
||||||
bool isConnected() { return connection->isConnected(); }
|
bool isConnected() { return connection->isConnected(); }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
PostgreSQLConnectionPtr connection;
|
ConnectionPtr connection;
|
||||||
Pool & pool;
|
Pool & pool;
|
||||||
};
|
};
|
||||||
|
|
||||||
using PostgreSQLConnectionHolderPtr = std::shared_ptr<PostgreSQLConnectionHolder>;
|
using ConnectionHolderPtr = std::shared_ptr<ConnectionHolder>;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -10,10 +10,10 @@
|
|||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace postgres
|
||||||
{
|
{
|
||||||
|
|
||||||
PostgreSQLConnectionPool::PostgreSQLConnectionPool(
|
ConnectionPool::ConnectionPool(
|
||||||
std::string dbname,
|
std::string dbname,
|
||||||
std::string host,
|
std::string host,
|
||||||
UInt16 port,
|
UInt16 port,
|
||||||
@ -37,7 +37,7 @@ PostgreSQLConnectionPool::PostgreSQLConnectionPool(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLConnectionPool::PostgreSQLConnectionPool(const PostgreSQLConnectionPool & other)
|
ConnectionPool::ConnectionPool(const ConnectionPool & other)
|
||||||
: pool(std::make_shared<Pool>(other.pool_size))
|
: pool(std::make_shared<Pool>(other.pool_size))
|
||||||
, connection_str(other.connection_str)
|
, connection_str(other.connection_str)
|
||||||
, address(other.address)
|
, address(other.address)
|
||||||
@ -49,46 +49,46 @@ PostgreSQLConnectionPool::PostgreSQLConnectionPool(const PostgreSQLConnectionPoo
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void PostgreSQLConnectionPool::initialize()
|
void ConnectionPool::initialize()
|
||||||
{
|
{
|
||||||
/// No connection is made, just fill pool with non-connected connection objects.
|
/// No connection is made, just fill pool with non-connected connection objects.
|
||||||
for (size_t i = 0; i < pool_size; ++i)
|
for (size_t i = 0; i < pool_size; ++i)
|
||||||
pool->push(std::make_shared<PostgreSQLConnection>(connection_str, address));
|
pool->push(std::make_shared<Connection>(connection_str, address));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
std::string PostgreSQLConnectionPool::formatConnectionString(
|
std::string ConnectionPool::formatConnectionString(
|
||||||
std::string dbname, std::string host, UInt16 port, std::string user, std::string password)
|
std::string dbname, std::string host, UInt16 port, std::string user, std::string password)
|
||||||
{
|
{
|
||||||
WriteBufferFromOwnString out;
|
DB::WriteBufferFromOwnString out;
|
||||||
out << "dbname=" << quote << dbname
|
out << "dbname=" << DB::quote << dbname
|
||||||
<< " host=" << quote << host
|
<< " host=" << DB::quote << host
|
||||||
<< " port=" << port
|
<< " port=" << port
|
||||||
<< " user=" << quote << user
|
<< " user=" << DB::quote << user
|
||||||
<< " password=" << quote << password;
|
<< " password=" << DB::quote << password;
|
||||||
return out.str();
|
return out.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLConnectionHolderPtr PostgreSQLConnectionPool::get()
|
ConnectionHolderPtr ConnectionPool::get()
|
||||||
{
|
{
|
||||||
PostgreSQLConnectionPtr connection;
|
ConnectionPtr connection;
|
||||||
|
|
||||||
/// Always blocks by default.
|
/// Always blocks by default.
|
||||||
if (block_on_empty_pool)
|
if (block_on_empty_pool)
|
||||||
{
|
{
|
||||||
/// pop to ConcurrentBoundedQueue will block until it is non-empty.
|
/// pop to ConcurrentBoundedQueue will block until it is non-empty.
|
||||||
pool->pop(connection);
|
pool->pop(connection);
|
||||||
return std::make_shared<PostgreSQLConnectionHolder>(connection, *pool);
|
return std::make_shared<ConnectionHolder>(connection, *pool);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (pool->tryPop(connection, pool_wait_timeout))
|
if (pool->tryPop(connection, pool_wait_timeout))
|
||||||
{
|
{
|
||||||
return std::make_shared<PostgreSQLConnectionHolder>(connection, *pool);
|
return std::make_shared<ConnectionHolder>(connection, *pool);
|
||||||
}
|
}
|
||||||
|
|
||||||
connection = std::make_shared<PostgreSQLConnection>(connection_str, address);
|
connection = std::make_shared<Connection>(connection_str, address);
|
||||||
return std::make_shared<PostgreSQLConnectionHolder>(connection, *pool);
|
return std::make_shared<ConnectionHolder>(connection, *pool);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -8,26 +8,25 @@
|
|||||||
#include "PostgreSQLConnection.h"
|
#include "PostgreSQLConnection.h"
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace postgres
|
||||||
{
|
{
|
||||||
|
|
||||||
class PostgreSQLReplicaConnection;
|
class PoolWithFailover;
|
||||||
|
|
||||||
|
|
||||||
/// Connection pool size is defined by user with setting `postgresql_connection_pool_size` (default 16).
|
/// Connection pool size is defined by user with setting `postgresql_connection_pool_size` (default 16).
|
||||||
/// If pool is empty, it will block until there are available connections.
|
/// If pool is empty, it will block until there are available connections.
|
||||||
/// If setting `connection_pool_wait_timeout` is defined, it will not block on empty pool and will
|
/// If setting `connection_pool_wait_timeout` is defined, it will not block on empty pool and will
|
||||||
/// wait until the timeout and then create a new connection. (only for storage/db engine)
|
/// wait until the timeout and then create a new connection. (only for storage/db engine)
|
||||||
class PostgreSQLConnectionPool
|
class ConnectionPool
|
||||||
{
|
{
|
||||||
|
|
||||||
friend class PostgreSQLReplicaConnection;
|
friend class PoolWithFailover;
|
||||||
|
|
||||||
static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16;
|
static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
|
|
||||||
PostgreSQLConnectionPool(
|
ConnectionPool(
|
||||||
std::string dbname,
|
std::string dbname,
|
||||||
std::string host,
|
std::string host,
|
||||||
UInt16 port,
|
UInt16 port,
|
||||||
@ -36,14 +35,14 @@ public:
|
|||||||
size_t pool_size_ = POSTGRESQL_POOL_DEFAULT_SIZE,
|
size_t pool_size_ = POSTGRESQL_POOL_DEFAULT_SIZE,
|
||||||
int64_t pool_wait_timeout_ = -1);
|
int64_t pool_wait_timeout_ = -1);
|
||||||
|
|
||||||
PostgreSQLConnectionPool(const PostgreSQLConnectionPool & other);
|
ConnectionPool(const ConnectionPool & other);
|
||||||
|
|
||||||
PostgreSQLConnectionPool operator =(const PostgreSQLConnectionPool &) = delete;
|
ConnectionPool operator =(const ConnectionPool &) = delete;
|
||||||
|
|
||||||
PostgreSQLConnectionHolderPtr get();
|
ConnectionHolderPtr get();
|
||||||
|
|
||||||
private:
|
private:
|
||||||
using Pool = ConcurrentBoundedQueue<PostgreSQLConnectionPtr>;
|
using Pool = ConcurrentBoundedQueue<ConnectionPtr>;
|
||||||
using PoolPtr = std::shared_ptr<Pool>;
|
using PoolPtr = std::shared_ptr<Pool>;
|
||||||
|
|
||||||
static std::string formatConnectionString(
|
static std::string formatConnectionString(
|
||||||
@ -58,7 +57,7 @@ private:
|
|||||||
bool block_on_empty_pool;
|
bool block_on_empty_pool;
|
||||||
};
|
};
|
||||||
|
|
||||||
using PostgreSQLConnectionPoolPtr = std::shared_ptr<PostgreSQLConnectionPool>;
|
using ConnectionPoolPtr = std::shared_ptr<ConnectionPool>;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
110
src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.cpp
Normal file
110
src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.cpp
Normal file
@ -0,0 +1,110 @@
|
|||||||
|
#include "PostgreSQLPoolWithFailover.h"
|
||||||
|
#include "PostgreSQLConnection.h"
|
||||||
|
#include <Common/parseRemoteDescription.h>
|
||||||
|
#include <Common/Exception.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int POSTGRESQL_CONNECTION_FAILURE;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace postgres
|
||||||
|
{
|
||||||
|
|
||||||
|
PoolWithFailover::PoolWithFailover(
|
||||||
|
const Poco::Util::AbstractConfiguration & config,
|
||||||
|
const std::string & config_prefix,
|
||||||
|
const size_t max_tries_)
|
||||||
|
: max_tries(max_tries_)
|
||||||
|
{
|
||||||
|
auto db = config.getString(config_prefix + ".db", "");
|
||||||
|
auto host = config.getString(config_prefix + ".host", "");
|
||||||
|
auto port = config.getUInt(config_prefix + ".port", 0);
|
||||||
|
auto user = config.getString(config_prefix + ".user", "");
|
||||||
|
auto password = config.getString(config_prefix + ".password", "");
|
||||||
|
|
||||||
|
if (config.has(config_prefix + ".replica"))
|
||||||
|
{
|
||||||
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||||
|
config.keys(config_prefix, config_keys);
|
||||||
|
|
||||||
|
for (const auto & config_key : config_keys)
|
||||||
|
{
|
||||||
|
if (config_key.starts_with("replica"))
|
||||||
|
{
|
||||||
|
std::string replica_name = config_prefix + "." + config_key;
|
||||||
|
size_t priority = config.getInt(replica_name + ".priority", 0);
|
||||||
|
|
||||||
|
auto replica_host = config.getString(replica_name + ".host", host);
|
||||||
|
auto replica_port = config.getUInt(replica_name + ".port", port);
|
||||||
|
auto replica_user = config.getString(replica_name + ".user", user);
|
||||||
|
auto replica_password = config.getString(replica_name + ".password", password);
|
||||||
|
|
||||||
|
replicas_with_priority[priority].emplace_back(std::make_shared<ConnectionPool>(db, replica_host, replica_port, replica_user, replica_password));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
replicas_with_priority[0].emplace_back(std::make_shared<ConnectionPool>(db, host, port, user, password));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
PoolWithFailover::PoolWithFailover(
|
||||||
|
const std::string & database,
|
||||||
|
const RemoteDescription & addresses,
|
||||||
|
const std::string & user,
|
||||||
|
const std::string & password,
|
||||||
|
size_t pool_size,
|
||||||
|
int64_t pool_wait_timeout,
|
||||||
|
size_t max_tries_)
|
||||||
|
: max_tries(max_tries_)
|
||||||
|
{
|
||||||
|
/// Replicas have the same priority, but traversed replicas are moved to the end of the queue.
|
||||||
|
for (const auto & [host, port] : addresses)
|
||||||
|
{
|
||||||
|
LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port);
|
||||||
|
replicas_with_priority[0].emplace_back(std::make_shared<ConnectionPool>(database, host, port, user, password, pool_size, pool_wait_timeout));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
|
||||||
|
: replicas_with_priority(other.replicas_with_priority)
|
||||||
|
, max_tries(other.max_tries)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ConnectionHolderPtr PoolWithFailover::get()
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
|
||||||
|
for (size_t try_idx = 0; try_idx < max_tries; ++try_idx)
|
||||||
|
{
|
||||||
|
for (auto & priority : replicas_with_priority)
|
||||||
|
{
|
||||||
|
auto & replicas = priority.second;
|
||||||
|
for (size_t i = 0; i < replicas.size(); ++i)
|
||||||
|
{
|
||||||
|
auto connection = replicas[i]->get();
|
||||||
|
if (connection->isConnected())
|
||||||
|
{
|
||||||
|
/// Move all traversed replicas to the end.
|
||||||
|
std::rotate(replicas.begin(), replicas.begin() + i + 1, replicas.end());
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
throw DB::Exception(DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, "Unable to connect to any of the replicas");
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
51
src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.h
Normal file
51
src/Storages/PostgreSQL/PostgreSQLPoolWithFailover.h
Normal file
@ -0,0 +1,51 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Core/Types.h>
|
||||||
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include "PostgreSQLConnectionPool.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace postgres
|
||||||
|
{
|
||||||
|
|
||||||
|
class PoolWithFailover
|
||||||
|
{
|
||||||
|
|
||||||
|
using RemoteDescription = std::vector<std::pair<String, uint16_t>>;
|
||||||
|
|
||||||
|
public:
|
||||||
|
static constexpr inline auto POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES = 5;
|
||||||
|
static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16;
|
||||||
|
|
||||||
|
PoolWithFailover(
|
||||||
|
const Poco::Util::AbstractConfiguration & config,
|
||||||
|
const std::string & config_prefix,
|
||||||
|
const size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
||||||
|
|
||||||
|
PoolWithFailover(
|
||||||
|
const std::string & database,
|
||||||
|
const RemoteDescription & addresses,
|
||||||
|
const std::string & user,
|
||||||
|
const std::string & password,
|
||||||
|
size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE,
|
||||||
|
int64_t pool_wait_timeout = -1,
|
||||||
|
size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
|
||||||
|
|
||||||
|
PoolWithFailover(const PoolWithFailover & other);
|
||||||
|
|
||||||
|
ConnectionHolderPtr get();
|
||||||
|
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Highest priority is 0, the bigger the number in map, the less the priority
|
||||||
|
using Replicas = std::vector<ConnectionPoolPtr>;
|
||||||
|
using ReplicasWithPriority = std::map<size_t, Replicas>;
|
||||||
|
|
||||||
|
ReplicasWithPriority replicas_with_priority;
|
||||||
|
size_t max_tries;
|
||||||
|
std::mutex mutex;
|
||||||
|
};
|
||||||
|
|
||||||
|
using PoolWithFailoverPtr = std::shared_ptr<PoolWithFailover>;
|
||||||
|
|
||||||
|
}
|
@ -1,79 +0,0 @@
|
|||||||
#include "PostgreSQLReplicaConnection.h"
|
|
||||||
#include "PostgreSQLConnection.h"
|
|
||||||
#include <Common/Exception.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int POSTGRESQL_CONNECTION_FAILURE;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLReplicaConnection::PostgreSQLReplicaConnection(
|
|
||||||
const Poco::Util::AbstractConfiguration & config,
|
|
||||||
const String & config_prefix,
|
|
||||||
const size_t num_retries_)
|
|
||||||
: num_retries(num_retries_)
|
|
||||||
{
|
|
||||||
auto db = config.getString(config_prefix + ".db", "");
|
|
||||||
auto host = config.getString(config_prefix + ".host", "");
|
|
||||||
auto port = config.getUInt(config_prefix + ".port", 0);
|
|
||||||
auto user = config.getString(config_prefix + ".user", "");
|
|
||||||
auto password = config.getString(config_prefix + ".password", "");
|
|
||||||
|
|
||||||
if (config.has(config_prefix + ".replica"))
|
|
||||||
{
|
|
||||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
||||||
config.keys(config_prefix, config_keys);
|
|
||||||
|
|
||||||
for (const auto & config_key : config_keys)
|
|
||||||
{
|
|
||||||
if (config_key.starts_with("replica"))
|
|
||||||
{
|
|
||||||
std::string replica_name = config_prefix + "." + config_key;
|
|
||||||
size_t priority = config.getInt(replica_name + ".priority", 0);
|
|
||||||
|
|
||||||
auto replica_host = config.getString(replica_name + ".host", host);
|
|
||||||
auto replica_port = config.getUInt(replica_name + ".port", port);
|
|
||||||
auto replica_user = config.getString(replica_name + ".user", user);
|
|
||||||
auto replica_password = config.getString(replica_name + ".password", password);
|
|
||||||
|
|
||||||
replicas[priority] = std::make_shared<PostgreSQLConnectionPool>(db, replica_host, replica_port, replica_user, replica_password);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
replicas[0] = std::make_shared<PostgreSQLConnectionPool>(db, host, port, user, password);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLReplicaConnection::PostgreSQLReplicaConnection(const PostgreSQLReplicaConnection & other)
|
|
||||||
: replicas(other.replicas)
|
|
||||||
, num_retries(other.num_retries)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
PostgreSQLConnectionHolderPtr PostgreSQLReplicaConnection::get()
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
|
|
||||||
for (size_t i = 0; i < num_retries; ++i)
|
|
||||||
{
|
|
||||||
for (auto & replica : replicas)
|
|
||||||
{
|
|
||||||
auto connection = replica.second->get();
|
|
||||||
if (connection->isConnected())
|
|
||||||
return connection;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
throw Exception(ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, "Unable to connect to any of the replicas");
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,8 +1,11 @@
|
|||||||
#include <Storages/StorageDistributed.h>
|
#include <Storages/StorageDistributed.h>
|
||||||
|
|
||||||
#include <Databases/IDatabase.h>
|
#include <Databases/IDatabase.h>
|
||||||
|
|
||||||
#include <Disks/IDisk.h>
|
#include <Disks/IDisk.h>
|
||||||
|
|
||||||
|
#include <DataStreams/RemoteBlockInputStream.h>
|
||||||
|
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <DataTypes/DataTypeUUID.h>
|
#include <DataTypes/DataTypeUUID.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
@ -31,9 +34,7 @@
|
|||||||
#include <Parsers/ParserAlterQuery.h>
|
#include <Parsers/ParserAlterQuery.h>
|
||||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
|
#include <Parsers/queryToString.h>
|
||||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
|
||||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
|
||||||
|
|
||||||
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
|
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
|
||||||
#include <Interpreters/ClusterProxy/executeQuery.h>
|
#include <Interpreters/ClusterProxy/executeQuery.h>
|
||||||
@ -42,6 +43,7 @@
|
|||||||
#include <Interpreters/InterpreterAlterQuery.h>
|
#include <Interpreters/InterpreterAlterQuery.h>
|
||||||
#include <Interpreters/InterpreterDescribeQuery.h>
|
#include <Interpreters/InterpreterDescribeQuery.h>
|
||||||
#include <Interpreters/InterpreterSelectQuery.h>
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
|
#include <Interpreters/JoinedTables.h>
|
||||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
@ -51,8 +53,12 @@
|
|||||||
#include <Interpreters/getTableExpressions.h>
|
#include <Interpreters/getTableExpressions.h>
|
||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
|
|
||||||
|
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||||
|
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||||
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
||||||
#include <Processors/Sources/NullSource.h>
|
#include <Processors/Sources/NullSource.h>
|
||||||
|
#include <Processors/Sources/SourceFromInputStream.h>
|
||||||
|
#include <Processors/NullSink.h>
|
||||||
|
|
||||||
#include <Core/Field.h>
|
#include <Core/Field.h>
|
||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
@ -60,6 +66,7 @@
|
|||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
|
#include <IO/ConnectionTimeoutsContext.h>
|
||||||
|
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
|
||||||
@ -75,6 +82,8 @@ const UInt64 FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_HAS_SHARDING_KEY = 1;
|
|||||||
const UInt64 FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_ALWAYS = 2;
|
const UInt64 FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_ALWAYS = 2;
|
||||||
|
|
||||||
const UInt64 DISTRIBUTED_GROUP_BY_NO_MERGE_AFTER_AGGREGATION = 2;
|
const UInt64 DISTRIBUTED_GROUP_BY_NO_MERGE_AFTER_AGGREGATION = 2;
|
||||||
|
|
||||||
|
const UInt64 PARALLEL_DISTRIBUTED_INSERT_SELECT_ALL = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
@ -613,6 +622,85 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
QueryPipelinePtr StorageDistributed::distributedWrite(const ASTInsertQuery & query, const Context & context)
|
||||||
|
{
|
||||||
|
const Settings & settings = context.getSettingsRef();
|
||||||
|
std::shared_ptr<StorageDistributed> storage_src;
|
||||||
|
auto & select = query.select->as<ASTSelectWithUnionQuery &>();
|
||||||
|
auto new_query = std::dynamic_pointer_cast<ASTInsertQuery>(query.clone());
|
||||||
|
if (select.list_of_selects->children.size() == 1)
|
||||||
|
{
|
||||||
|
if (auto * select_query = select.list_of_selects->children.at(0)->as<ASTSelectQuery>())
|
||||||
|
{
|
||||||
|
JoinedTables joined_tables(Context(context), *select_query);
|
||||||
|
|
||||||
|
if (joined_tables.tablesCount() == 1)
|
||||||
|
{
|
||||||
|
storage_src = std::dynamic_pointer_cast<StorageDistributed>(joined_tables.getLeftTableStorage());
|
||||||
|
if (storage_src)
|
||||||
|
{
|
||||||
|
const auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
|
||||||
|
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
|
||||||
|
|
||||||
|
auto new_select_query = std::dynamic_pointer_cast<ASTSelectQuery>(select_query->clone());
|
||||||
|
select_with_union_query->list_of_selects->children.push_back(new_select_query);
|
||||||
|
|
||||||
|
new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName());
|
||||||
|
|
||||||
|
new_query->select = select_with_union_query;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!storage_src || storage_src->getClusterName() != getClusterName())
|
||||||
|
{
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (settings.parallel_distributed_insert_select == PARALLEL_DISTRIBUTED_INSERT_SELECT_ALL)
|
||||||
|
{
|
||||||
|
new_query->table_id = StorageID(getRemoteDatabaseName(), getRemoteTableName());
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto & cluster = getCluster();
|
||||||
|
const auto & shards_info = cluster->getShardsInfo();
|
||||||
|
|
||||||
|
std::vector<std::unique_ptr<QueryPipeline>> pipelines;
|
||||||
|
|
||||||
|
String new_query_str = queryToString(new_query);
|
||||||
|
for (size_t shard_index : ext::range(0, shards_info.size()))
|
||||||
|
{
|
||||||
|
const auto & shard_info = shards_info[shard_index];
|
||||||
|
if (shard_info.isLocal())
|
||||||
|
{
|
||||||
|
InterpreterInsertQuery interpreter(new_query, context);
|
||||||
|
pipelines.emplace_back(std::make_unique<QueryPipeline>(interpreter.execute().pipeline));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
|
||||||
|
auto connections = shard_info.pool->getMany(timeouts, &settings, PoolMode::GET_ONE);
|
||||||
|
if (connections.empty() || connections.front().isNull())
|
||||||
|
throw Exception(
|
||||||
|
"Expected exactly one connection for shard " + toString(shard_info.shard_num), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
/// INSERT SELECT query returns empty block
|
||||||
|
auto in_stream = std::make_shared<RemoteBlockInputStream>(std::move(connections), new_query_str, Block{}, context);
|
||||||
|
pipelines.emplace_back(std::make_unique<QueryPipeline>());
|
||||||
|
pipelines.back()->init(Pipe(std::make_shared<SourceFromInputStream>(std::move(in_stream))));
|
||||||
|
pipelines.back()->setSinks([](const Block & header, QueryPipeline::StreamType) -> ProcessorPtr
|
||||||
|
{
|
||||||
|
return std::make_shared<EmptySink>(header);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return std::make_unique<QueryPipeline>(
|
||||||
|
QueryPipeline::unitePipelines(std::move(pipelines), {}, ExpressionActionsSettings::fromContext(context)));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||||
{
|
{
|
||||||
auto name_deps = getDependentViewsByColumn(context);
|
auto name_deps = getDependentViewsByColumn(context);
|
||||||
|
@ -81,6 +81,8 @@ public:
|
|||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override;
|
||||||
|
|
||||||
|
QueryPipelinePtr distributedWrite(const ASTInsertQuery & query, const Context & context) override;
|
||||||
|
|
||||||
/// Removes temporary data in local filesystem.
|
/// Removes temporary data in local filesystem.
|
||||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
|
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
|
||||||
|
|
||||||
|
189
src/Storages/StorageExternalDistributed.cpp
Normal file
189
src/Storages/StorageExternalDistributed.cpp
Normal file
@ -0,0 +1,189 @@
|
|||||||
|
#include "StorageExternalDistributed.h"
|
||||||
|
|
||||||
|
#if USE_MYSQL || USE_LIBPQXX
|
||||||
|
|
||||||
|
#include <Storages/StorageFactory.h>
|
||||||
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
|
#include <Core/Settings.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
#include <Common/parseAddress.h>
|
||||||
|
#include <Processors/Pipe.h>
|
||||||
|
#include <Common/parseRemoteDescription.h>
|
||||||
|
#include <Storages/StorageMySQL.h>
|
||||||
|
#include <Storages/StoragePostgreSQL.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
|
}
|
||||||
|
|
||||||
|
StorageExternalDistributed::StorageExternalDistributed(
|
||||||
|
const StorageID & table_id_,
|
||||||
|
ExternalStorageEngine table_engine,
|
||||||
|
const String & cluster_description,
|
||||||
|
const String & remote_database,
|
||||||
|
const String & remote_table,
|
||||||
|
const String & username,
|
||||||
|
const String & password,
|
||||||
|
const ColumnsDescription & columns_,
|
||||||
|
const ConstraintsDescription & constraints_,
|
||||||
|
const Context & context)
|
||||||
|
: IStorage(table_id_)
|
||||||
|
{
|
||||||
|
StorageInMemoryMetadata storage_metadata;
|
||||||
|
storage_metadata.setColumns(columns_);
|
||||||
|
storage_metadata.setConstraints(constraints_);
|
||||||
|
setInMemoryMetadata(storage_metadata);
|
||||||
|
|
||||||
|
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
|
||||||
|
std::vector<String> shards_descriptions = parseRemoteDescription(cluster_description, 0, cluster_description.size(), ',', max_addresses);
|
||||||
|
std::vector<std::pair<std::string, UInt16>> addresses;
|
||||||
|
|
||||||
|
/// For each shard pass replicas description into storage, replicas are managed by storage's PoolWithFailover.
|
||||||
|
for (const auto & shard_description : shards_descriptions)
|
||||||
|
{
|
||||||
|
StoragePtr shard;
|
||||||
|
|
||||||
|
switch (table_engine)
|
||||||
|
{
|
||||||
|
#if USE_MYSQL
|
||||||
|
case ExternalStorageEngine::MySQL:
|
||||||
|
{
|
||||||
|
addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 3306);
|
||||||
|
|
||||||
|
mysqlxx::PoolWithFailover pool(
|
||||||
|
remote_database,
|
||||||
|
addresses,
|
||||||
|
username, password);
|
||||||
|
|
||||||
|
shard = StorageMySQL::create(
|
||||||
|
table_id_,
|
||||||
|
std::move(pool),
|
||||||
|
remote_database,
|
||||||
|
remote_table,
|
||||||
|
/* replace_query = */ false,
|
||||||
|
/* on_duplicate_clause = */ "",
|
||||||
|
columns_, constraints_,
|
||||||
|
context);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
#if USE_LIBPQXX
|
||||||
|
|
||||||
|
case ExternalStorageEngine::PostgreSQL:
|
||||||
|
{
|
||||||
|
addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 5432);
|
||||||
|
|
||||||
|
postgres::PoolWithFailover pool(
|
||||||
|
remote_database,
|
||||||
|
addresses,
|
||||||
|
username, password,
|
||||||
|
context.getSettingsRef().postgresql_connection_pool_size,
|
||||||
|
context.getSettingsRef().postgresql_connection_pool_wait_timeout);
|
||||||
|
|
||||||
|
shard = StoragePostgreSQL::create(
|
||||||
|
table_id_,
|
||||||
|
std::move(pool),
|
||||||
|
remote_table,
|
||||||
|
columns_, constraints_,
|
||||||
|
context);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
default:
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||||
|
"Unsupported table engine. Supported engines are: MySQL, PostgreSQL");
|
||||||
|
}
|
||||||
|
|
||||||
|
shards.emplace(std::move(shard));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Pipe StorageExternalDistributed::read(
|
||||||
|
const Names & column_names,
|
||||||
|
const StorageMetadataPtr & metadata_snapshot,
|
||||||
|
SelectQueryInfo & query_info,
|
||||||
|
const Context & context,
|
||||||
|
QueryProcessingStage::Enum processed_stage,
|
||||||
|
size_t max_block_size,
|
||||||
|
unsigned num_streams)
|
||||||
|
{
|
||||||
|
Pipes pipes;
|
||||||
|
for (const auto & shard : shards)
|
||||||
|
{
|
||||||
|
pipes.emplace_back(shard->read(
|
||||||
|
column_names,
|
||||||
|
metadata_snapshot,
|
||||||
|
query_info,
|
||||||
|
context,
|
||||||
|
processed_stage,
|
||||||
|
max_block_size,
|
||||||
|
num_streams
|
||||||
|
));
|
||||||
|
}
|
||||||
|
|
||||||
|
return Pipe::unitePipes(std::move(pipes));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void registerStorageExternalDistributed(StorageFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerStorage("ExternalDistributed", [](const StorageFactory::Arguments & args)
|
||||||
|
{
|
||||||
|
ASTs & engine_args = args.engine_args;
|
||||||
|
|
||||||
|
if (engine_args.size() != 6)
|
||||||
|
throw Exception(
|
||||||
|
"Storage MySQLiDistributed requires 5 parameters: ExternalDistributed('engine_name', 'cluster_description', database, table, 'user', 'password').",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
for (auto & engine_arg : engine_args)
|
||||||
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
|
||||||
|
|
||||||
|
const String & engine_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & cluster_description = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & remote_database = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & remote_table = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & username = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & password = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
|
||||||
|
StorageExternalDistributed::ExternalStorageEngine table_engine;
|
||||||
|
if (engine_name == "MySQL")
|
||||||
|
table_engine = StorageExternalDistributed::ExternalStorageEngine::MySQL;
|
||||||
|
else if (engine_name == "PostgreSQL")
|
||||||
|
table_engine = StorageExternalDistributed::ExternalStorageEngine::PostgreSQL;
|
||||||
|
else
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||||
|
"External storage engine {} is not supported for StorageExternalDistributed. Supported engines are: MySQL, PostgreSQL",
|
||||||
|
engine_name);
|
||||||
|
|
||||||
|
return StorageExternalDistributed::create(
|
||||||
|
args.table_id,
|
||||||
|
table_engine,
|
||||||
|
cluster_description,
|
||||||
|
remote_database,
|
||||||
|
remote_table,
|
||||||
|
username,
|
||||||
|
password,
|
||||||
|
args.columns,
|
||||||
|
args.constraints,
|
||||||
|
args.context);
|
||||||
|
},
|
||||||
|
{
|
||||||
|
.source_access_type = AccessType::MYSQL,
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
65
src/Storages/StorageExternalDistributed.h
Normal file
65
src/Storages/StorageExternalDistributed.h
Normal file
@ -0,0 +1,65 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
#include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_MYSQL || USE_LIBPQXX
|
||||||
|
|
||||||
|
#include <ext/shared_ptr_helper.h>
|
||||||
|
#include <Storages/IStorage.h>
|
||||||
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas.
|
||||||
|
/// This class unites multiple storages with replicas into multiple shards with replicas.
|
||||||
|
/// A query to external database is passed to one replica on each shard, the result is united.
|
||||||
|
/// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue.
|
||||||
|
/// TODO: try `load_balancing` setting for replicas priorities same way as for table function `remote`
|
||||||
|
class StorageExternalDistributed final : public ext::shared_ptr_helper<StorageExternalDistributed>, public DB::IStorage
|
||||||
|
{
|
||||||
|
friend struct ext::shared_ptr_helper<StorageExternalDistributed>;
|
||||||
|
|
||||||
|
public:
|
||||||
|
enum class ExternalStorageEngine
|
||||||
|
{
|
||||||
|
MySQL,
|
||||||
|
PostgreSQL,
|
||||||
|
Default
|
||||||
|
};
|
||||||
|
|
||||||
|
std::string getName() const override { return "ExternalDistributed"; }
|
||||||
|
|
||||||
|
Pipe read(
|
||||||
|
const Names & column_names,
|
||||||
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||||
|
SelectQueryInfo & query_info,
|
||||||
|
const Context & context,
|
||||||
|
QueryProcessingStage::Enum processed_stage,
|
||||||
|
size_t max_block_size,
|
||||||
|
unsigned num_streams) override;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
StorageExternalDistributed(
|
||||||
|
const StorageID & table_id_,
|
||||||
|
ExternalStorageEngine table_engine,
|
||||||
|
const String & cluster_description,
|
||||||
|
const String & remote_database_,
|
||||||
|
const String & remote_table_,
|
||||||
|
const String & username,
|
||||||
|
const String & password,
|
||||||
|
const ColumnsDescription & columns_,
|
||||||
|
const ConstraintsDescription & constraints_,
|
||||||
|
const Context & context_);
|
||||||
|
|
||||||
|
private:
|
||||||
|
using Shards = std::unordered_set<StoragePtr>;
|
||||||
|
Shards shards;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
@ -18,6 +18,7 @@
|
|||||||
#include <mysqlxx/Transaction.h>
|
#include <mysqlxx/Transaction.h>
|
||||||
#include <Processors/Sources/SourceFromInputStream.h>
|
#include <Processors/Sources/SourceFromInputStream.h>
|
||||||
#include <Processors/Pipe.h>
|
#include <Processors/Pipe.h>
|
||||||
|
#include <Common/parseRemoteDescription.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -41,7 +42,7 @@ static String backQuoteMySQL(const String & x)
|
|||||||
|
|
||||||
StorageMySQL::StorageMySQL(
|
StorageMySQL::StorageMySQL(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
mysqlxx::Pool && pool_,
|
mysqlxx::PoolWithFailover && pool_,
|
||||||
const std::string & remote_database_name_,
|
const std::string & remote_database_name_,
|
||||||
const std::string & remote_table_name_,
|
const std::string & remote_table_name_,
|
||||||
const bool replace_query_,
|
const bool replace_query_,
|
||||||
@ -54,7 +55,7 @@ StorageMySQL::StorageMySQL(
|
|||||||
, remote_table_name(remote_table_name_)
|
, remote_table_name(remote_table_name_)
|
||||||
, replace_query{replace_query_}
|
, replace_query{replace_query_}
|
||||||
, on_duplicate_clause{on_duplicate_clause_}
|
, on_duplicate_clause{on_duplicate_clause_}
|
||||||
, pool(std::move(pool_))
|
, pool(std::make_shared<mysqlxx::PoolWithFailover>(pool_))
|
||||||
, global_context(context_.getGlobalContext())
|
, global_context(context_.getGlobalContext())
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
@ -94,9 +95,8 @@ Pipe StorageMySQL::read(
|
|||||||
sample_block.insert({ column_data.type, column_data.name });
|
sample_block.insert({ column_data.type, column_data.name });
|
||||||
}
|
}
|
||||||
|
|
||||||
/// TODO: rewrite MySQLBlockInputStream
|
|
||||||
return Pipe(std::make_shared<SourceFromInputStream>(
|
return Pipe(std::make_shared<SourceFromInputStream>(
|
||||||
std::make_shared<MySQLLazyBlockInputStream>(pool, query, sample_block, max_block_size_, /* auto_close = */ true)));
|
std::make_shared<MySQLWithFailoverBlockInputStream>(pool, query, sample_block, max_block_size_, /* auto_close = */ true)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -213,7 +213,7 @@ private:
|
|||||||
|
|
||||||
BlockOutputStreamPtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context)
|
BlockOutputStreamPtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<StorageMySQLBlockOutputStream>(*this, metadata_snapshot, remote_database_name, remote_table_name, pool.get(), context.getSettingsRef().mysql_max_rows_to_insert);
|
return std::make_shared<StorageMySQLBlockOutputStream>(*this, metadata_snapshot, remote_database_name, remote_table_name, pool->get(), context.getSettingsRef().mysql_max_rows_to_insert);
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerStorageMySQL(StorageFactory & factory)
|
void registerStorageMySQL(StorageFactory & factory)
|
||||||
@ -224,21 +224,22 @@ void registerStorageMySQL(StorageFactory & factory)
|
|||||||
|
|
||||||
if (engine_args.size() < 5 || engine_args.size() > 7)
|
if (engine_args.size() < 5 || engine_args.size() > 7)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"Storage MySQL requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
|
"Storage MySQL requires 5-7 parameters: MySQL('host:port' (or 'addresses_pattern'), database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
for (auto & engine_arg : engine_args)
|
for (auto & engine_arg : engine_args)
|
||||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
|
||||||
|
|
||||||
/// 3306 is the default MySQL port.
|
/// 3306 is the default MySQL port.
|
||||||
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 3306);
|
const String & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
|
||||||
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
size_t max_addresses = args.context.getSettingsRef().glob_expansion_max_elements;
|
||||||
|
|
||||||
mysqlxx::Pool pool(remote_database, parsed_host_port.first, username, password, parsed_host_port.second);
|
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
|
||||||
|
mysqlxx::PoolWithFailover pool(remote_database, addresses, username, password);
|
||||||
|
|
||||||
bool replace_query = false;
|
bool replace_query = false;
|
||||||
std::string on_duplicate_clause;
|
std::string on_duplicate_clause;
|
||||||
|
@ -1,15 +1,15 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#if !defined(ARCADIA_BUILD)
|
#if !defined(ARCADIA_BUILD)
|
||||||
# include "config_core.h"
|
#include "config_core.h"
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
|
|
||||||
# include <ext/shared_ptr_helper.h>
|
#include <ext/shared_ptr_helper.h>
|
||||||
|
|
||||||
# include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
# include <mysqlxx/Pool.h>
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -25,7 +25,7 @@ class StorageMySQL final : public ext::shared_ptr_helper<StorageMySQL>, public I
|
|||||||
public:
|
public:
|
||||||
StorageMySQL(
|
StorageMySQL(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
mysqlxx::Pool && pool_,
|
mysqlxx::PoolWithFailover && pool_,
|
||||||
const std::string & remote_database_name_,
|
const std::string & remote_database_name_,
|
||||||
const std::string & remote_table_name_,
|
const std::string & remote_table_name_,
|
||||||
const bool replace_query_,
|
const bool replace_query_,
|
||||||
@ -55,7 +55,7 @@ private:
|
|||||||
bool replace_query;
|
bool replace_query;
|
||||||
std::string on_duplicate_clause;
|
std::string on_duplicate_clause;
|
||||||
|
|
||||||
mysqlxx::Pool pool;
|
mysqlxx::PoolWithFailoverPtr pool;
|
||||||
const Context & global_context;
|
const Context & global_context;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
#include <Formats/FormatSettings.h>
|
#include <Formats/FormatSettings.h>
|
||||||
#include <Processors/Sources/SourceFromInputStream.h>
|
#include <Processors/Sources/SourceFromInputStream.h>
|
||||||
|
#include <Common/parseRemoteDescription.h>
|
||||||
#include <Processors/Pipe.h>
|
#include <Processors/Pipe.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
@ -41,8 +42,8 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
StoragePostgreSQL::StoragePostgreSQL(
|
StoragePostgreSQL::StoragePostgreSQL(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
|
const postgres::PoolWithFailover & pool_,
|
||||||
const String & remote_table_name_,
|
const String & remote_table_name_,
|
||||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
@ -51,7 +52,7 @@ StoragePostgreSQL::StoragePostgreSQL(
|
|||||||
, remote_table_name(remote_table_name_)
|
, remote_table_name(remote_table_name_)
|
||||||
, remote_table_schema(remote_table_schema_)
|
, remote_table_schema(remote_table_schema_)
|
||||||
, global_context(context_)
|
, global_context(context_)
|
||||||
, connection_pool(std::move(connection_pool_))
|
, pool(std::make_shared<postgres::PoolWithFailover>(pool_))
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
storage_metadata.setColumns(columns_);
|
storage_metadata.setColumns(columns_);
|
||||||
@ -88,7 +89,7 @@ Pipe StoragePostgreSQL::read(
|
|||||||
}
|
}
|
||||||
|
|
||||||
return Pipe(std::make_shared<SourceFromInputStream>(
|
return Pipe(std::make_shared<SourceFromInputStream>(
|
||||||
std::make_shared<PostgreSQLBlockInputStream>(connection_pool->get(), query, sample_block, max_block_size_)));
|
std::make_shared<PostgreSQLBlockInputStream>(pool->get(), query, sample_block, max_block_size_)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -97,7 +98,7 @@ class PostgreSQLBlockOutputStream : public IBlockOutputStream
|
|||||||
public:
|
public:
|
||||||
explicit PostgreSQLBlockOutputStream(
|
explicit PostgreSQLBlockOutputStream(
|
||||||
const StorageMetadataPtr & metadata_snapshot_,
|
const StorageMetadataPtr & metadata_snapshot_,
|
||||||
PostgreSQLConnectionHolderPtr connection_,
|
postgres::ConnectionHolderPtr connection_,
|
||||||
const std::string & remote_table_name_)
|
const std::string & remote_table_name_)
|
||||||
: metadata_snapshot(metadata_snapshot_)
|
: metadata_snapshot(metadata_snapshot_)
|
||||||
, connection(std::move(connection_))
|
, connection(std::move(connection_))
|
||||||
@ -276,7 +277,7 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
StorageMetadataPtr metadata_snapshot;
|
StorageMetadataPtr metadata_snapshot;
|
||||||
PostgreSQLConnectionHolderPtr connection;
|
postgres::ConnectionHolderPtr connection;
|
||||||
std::string remote_table_name;
|
std::string remote_table_name;
|
||||||
|
|
||||||
std::unique_ptr<pqxx::work> work;
|
std::unique_ptr<pqxx::work> work;
|
||||||
@ -287,7 +288,7 @@ private:
|
|||||||
BlockOutputStreamPtr StoragePostgreSQL::write(
|
BlockOutputStreamPtr StoragePostgreSQL::write(
|
||||||
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */)
|
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */)
|
||||||
{
|
{
|
||||||
return std::make_shared<PostgreSQLBlockOutputStream>(metadata_snapshot, connection_pool->get(), remote_table_name);
|
return std::make_shared<PostgreSQLBlockOutputStream>(metadata_snapshot, pool->get(), remote_table_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -305,24 +306,31 @@ void registerStoragePostgreSQL(StorageFactory & factory)
|
|||||||
for (auto & engine_arg : engine_args)
|
for (auto & engine_arg : engine_args)
|
||||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
|
||||||
|
|
||||||
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 5432);
|
auto host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
/// Split into replicas if needed.
|
||||||
|
size_t max_addresses = args.context.getSettingsRef().glob_expansion_max_elements;
|
||||||
|
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
|
||||||
|
|
||||||
|
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
|
||||||
String remote_table_schema;
|
String remote_table_schema;
|
||||||
if (engine_args.size() == 6)
|
if (engine_args.size() == 6)
|
||||||
remote_table_schema = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
remote_table_schema = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
|
||||||
auto connection_pool = std::make_shared<PostgreSQLConnectionPool>(
|
postgres::PoolWithFailover pool(
|
||||||
engine_args[1]->as<ASTLiteral &>().value.safeGet<String>(),
|
remote_database,
|
||||||
parsed_host_port.first,
|
addresses,
|
||||||
parsed_host_port.second,
|
username,
|
||||||
engine_args[3]->as<ASTLiteral &>().value.safeGet<String>(),
|
password,
|
||||||
engine_args[4]->as<ASTLiteral &>().value.safeGet<String>(),
|
|
||||||
args.context.getSettingsRef().postgresql_connection_pool_size,
|
args.context.getSettingsRef().postgresql_connection_pool_size,
|
||||||
args.context.getSettingsRef().postgresql_connection_pool_wait_timeout);
|
args.context.getSettingsRef().postgresql_connection_pool_wait_timeout);
|
||||||
|
|
||||||
return StoragePostgreSQL::create(
|
return StoragePostgreSQL::create(
|
||||||
args.table_id, remote_table, connection_pool, args.columns, args.constraints, args.context, remote_table_schema);
|
args.table_id, pool, remote_table,
|
||||||
|
args.columns, args.constraints, args.context, remote_table_schema);
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
.source_access_type = AccessType::POSTGRES,
|
.source_access_type = AccessType::POSTGRES,
|
||||||
|
@ -9,7 +9,7 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <DataStreams/IBlockOutputStream.h>
|
#include <DataStreams/IBlockOutputStream.h>
|
||||||
#include <Storages/PostgreSQL/PostgreSQLConnectionPool.h>
|
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
|
||||||
#include <pqxx/pqxx>
|
#include <pqxx/pqxx>
|
||||||
|
|
||||||
|
|
||||||
@ -23,8 +23,8 @@ class StoragePostgreSQL final : public ext::shared_ptr_helper<StoragePostgreSQL>
|
|||||||
public:
|
public:
|
||||||
StoragePostgreSQL(
|
StoragePostgreSQL(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
|
const postgres::PoolWithFailover & pool_,
|
||||||
const String & remote_table_name_,
|
const String & remote_table_name_,
|
||||||
PostgreSQLConnectionPoolPtr connection_pool_,
|
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
@ -49,7 +49,7 @@ private:
|
|||||||
String remote_table_name;
|
String remote_table_name;
|
||||||
String remote_table_schema;
|
String remote_table_schema;
|
||||||
Context global_context;
|
Context global_context;
|
||||||
PostgreSQLConnectionPoolPtr connection_pool;
|
postgres::PoolWithFailoverPtr pool;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -455,12 +455,12 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas(
|
|||||||
if (replicas.empty())
|
if (replicas.empty())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
zkutil::EventPtr wait_event = std::make_shared<Poco::Event>();
|
|
||||||
|
|
||||||
std::set<String> inactive_replicas;
|
std::set<String> inactive_replicas;
|
||||||
for (const String & replica : replicas)
|
for (const String & replica : replicas)
|
||||||
{
|
{
|
||||||
LOG_DEBUG(log, "Waiting for {} to apply mutation {}", replica, mutation_id);
|
LOG_DEBUG(log, "Waiting for {} to apply mutation {}", replica, mutation_id);
|
||||||
|
zkutil::EventPtr wait_event = std::make_shared<Poco::Event>();
|
||||||
|
|
||||||
while (!partial_shutdown_called)
|
while (!partial_shutdown_called)
|
||||||
{
|
{
|
||||||
@ -484,9 +484,8 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas(
|
|||||||
|
|
||||||
String mutation_pointer = zookeeper_path + "/replicas/" + replica + "/mutation_pointer";
|
String mutation_pointer = zookeeper_path + "/replicas/" + replica + "/mutation_pointer";
|
||||||
std::string mutation_pointer_value;
|
std::string mutation_pointer_value;
|
||||||
Coordination::Stat get_stat;
|
|
||||||
/// Replica could be removed
|
/// Replica could be removed
|
||||||
if (!zookeeper->tryGet(mutation_pointer, mutation_pointer_value, &get_stat, wait_event))
|
if (!zookeeper->tryGet(mutation_pointer, mutation_pointer_value, nullptr, wait_event))
|
||||||
{
|
{
|
||||||
LOG_WARNING(log, "Replica {} was removed", replica);
|
LOG_WARNING(log, "Replica {} was removed", replica);
|
||||||
break;
|
break;
|
||||||
@ -496,8 +495,10 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas(
|
|||||||
|
|
||||||
/// Replica can become inactive, so wait with timeout and recheck it
|
/// Replica can become inactive, so wait with timeout and recheck it
|
||||||
if (wait_event->tryWait(1000))
|
if (wait_event->tryWait(1000))
|
||||||
break;
|
continue;
|
||||||
|
|
||||||
|
/// Here we check mutation for errors or kill on local replica. If they happen on this replica
|
||||||
|
/// they will happen on each replica, so we can check only in-memory info.
|
||||||
auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id);
|
auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id);
|
||||||
if (!mutation_status || !mutation_status->latest_fail_reason.empty())
|
if (!mutation_status || !mutation_status->latest_fail_reason.empty())
|
||||||
break;
|
break;
|
||||||
@ -514,6 +515,8 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas(
|
|||||||
std::set<String> mutation_ids;
|
std::set<String> mutation_ids;
|
||||||
mutation_ids.insert(mutation_id);
|
mutation_ids.insert(mutation_id);
|
||||||
|
|
||||||
|
/// Here we check mutation for errors or kill on local replica. If they happen on this replica
|
||||||
|
/// they will happen on each replica, so we can check only in-memory info.
|
||||||
auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id, &mutation_ids);
|
auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id, &mutation_ids);
|
||||||
checkMutationStatus(mutation_status, mutation_ids);
|
checkMutationStatus(mutation_status, mutation_ids);
|
||||||
|
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Storages/StorageURL.h>
|
#include <Storages/StorageURL.h>
|
||||||
#include <Common/XDBCBridgeHelper.h>
|
#include <Bridge/XDBCBridgeHelper.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user