mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
CLICKHOUSE-3878: Remove connection string building and validation, change format of columns in request, more convinient exception messages
This commit is contained in:
parent
bff4bbf634
commit
6d40546a9a
@ -1,6 +1,5 @@
|
||||
#include "HandlerFactory.h"
|
||||
#include <Common/HTMLForm.h>
|
||||
#include "Handlers.h"
|
||||
|
||||
#include <Dictionaries/validateODBCConnectionString.h>
|
||||
#include <Poco/Ext/SessionPoolHelpers.h>
|
||||
@ -9,15 +8,6 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
std::string buildConnectionString(const std::string & DSN, const std::string & database)
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << "DSN=" << DSN << ";DATABASE=" << database;
|
||||
return ss.str();
|
||||
}
|
||||
}
|
||||
Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
|
||||
{
|
||||
const auto & uri = request.getURI();
|
||||
@ -26,49 +16,8 @@ Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco:
|
||||
if (uri == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
|
||||
return new PingHandler(keep_alive_timeout);
|
||||
|
||||
HTMLForm params(request);
|
||||
std::string DSN = params.get("DSN", "");
|
||||
std::string database = params.get("database", "");
|
||||
|
||||
std::string max_block_size_str = params.get("max_block_size", "");
|
||||
std::string format = params.get("format", "RowBinary");
|
||||
|
||||
std::string connection_string = buildConnectionString(DSN, database);
|
||||
|
||||
LOG_TRACE(log, "Connection string:" << connection_string);
|
||||
|
||||
std::shared_ptr<Poco::Data::SessionPool> pool = nullptr;
|
||||
if (!pool_map.count(connection_string))
|
||||
try
|
||||
{
|
||||
std::string validated = validateODBCConnectionString(connection_string);
|
||||
pool
|
||||
= createAndCheckResizePocoSessionPool([validated] { return std::make_shared<Poco::Data::SessionPool>("ODBC", validated); });
|
||||
pool_map[connection_string] = pool;
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
LOG_WARNING(log, "Connection string validation failed: " + ex.message());
|
||||
}
|
||||
else
|
||||
{
|
||||
pool = pool_map[connection_string];
|
||||
}
|
||||
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE;
|
||||
|
||||
if (!max_block_size_str.empty())
|
||||
try
|
||||
{
|
||||
max_block_size = std::stoul(max_block_size_str);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
|
||||
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
|
||||
return new ODBCHandler(pool, format, max_block_size, keep_alive_timeout, context);
|
||||
return new ODBCHandler(pool_map, keep_alive_timeout, context);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Net/HTTPRequestHandler.h>
|
||||
#include <Poco/Net/HTTPRequestHandlerFactory.h>
|
||||
#include "Handlers.h"
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
@ -18,6 +19,7 @@ public:
|
||||
HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, std::shared_ptr<Context> context_)
|
||||
: log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_)
|
||||
{
|
||||
pool_map = std::make_shared<ODBCHandler::PoolMap>();
|
||||
}
|
||||
|
||||
Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override;
|
||||
@ -27,6 +29,6 @@ private:
|
||||
std::string name;
|
||||
size_t keep_alive_timeout;
|
||||
std::shared_ptr<Context> context;
|
||||
std::unordered_map<std::string, std::shared_ptr<Poco::Data::SessionPool>> pool_map;
|
||||
std::shared_ptr<ODBCHandler::PoolMap> pool_map;
|
||||
};
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
|
||||
#include <memory>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Dictionaries/ODBCBlockInputStream.h>
|
||||
#include <Formats/BinaryRowInputStream.h>
|
||||
@ -13,6 +14,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <boost/tokenizer.hpp>
|
||||
#include <Poco/Ext/SessionPoolHelpers.h>
|
||||
#include <Poco/Net/HTTPServerRequest.h>
|
||||
#include <Poco/Net/HTTPServerResponse.h>
|
||||
#include <common/logger_useful.h>
|
||||
@ -25,32 +27,46 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
std::optional<NamesAndTypesList> parseColumns(std::string && column_string, Poco::Logger * log)
|
||||
std::unique_ptr<Block> parseColumns(std::string && column_string)
|
||||
{
|
||||
const auto & factory_instance = DataTypeFactory::instance();
|
||||
NamesAndTypesList result;
|
||||
static boost::char_separator<char> sep(",");
|
||||
boost::tokenizer<boost::char_separator<char>> tokens(column_string, sep);
|
||||
for (const std::string & name_and_type_str : tokens)
|
||||
std::unique_ptr<Block> sample_block = std::make_unique<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;
|
||||
}
|
||||
|
||||
size_t parseMaxBlockSize(const std::string & max_block_size_str, Poco::Logger * log)
|
||||
{
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE;
|
||||
if (!max_block_size_str.empty())
|
||||
{
|
||||
std::vector<std::string> name_and_type;
|
||||
boost::split(name_and_type, name_and_type_str, boost::is_any_of(":"));
|
||||
if (name_and_type.size() != 2)
|
||||
return std::nullopt;
|
||||
try
|
||||
{
|
||||
result.emplace_back(name_and_type[0], factory_instance.get(name_and_type[1]));
|
||||
max_block_size = std::stoul(max_block_size_str);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
return std::nullopt;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
return max_block_size;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ODBCHandler::PoolPtr ODBCHandler::getPool(const std::string & connection_str)
|
||||
{
|
||||
if (!pool_map->count(connection_str))
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
pool_map->emplace(connection_str, createAndCheckResizePocoSessionPool([connection_str] {
|
||||
return std::make_shared<Poco::Data::SessionPool>("ODBC", connection_str);
|
||||
}));
|
||||
}
|
||||
return pool_map->at(connection_str);
|
||||
}
|
||||
|
||||
void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
Poco::Net::HTMLForm params(request, request.stream());
|
||||
@ -63,12 +79,6 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
LOG_WARNING(log, message);
|
||||
};
|
||||
|
||||
if (pool == nullptr)
|
||||
{
|
||||
process_error("ODBCBridge: DSN or database in URL params is not provided or incorrect");
|
||||
return;
|
||||
}
|
||||
|
||||
if (!params.has("query"))
|
||||
{
|
||||
process_error("ODBCBridge: No 'query' in request body");
|
||||
@ -77,45 +87,49 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
|
||||
if (!params.has("columns"))
|
||||
{
|
||||
process_error("ODBCBridge: No 'columns' in request body");
|
||||
process_error("ODBCBridge: No 'columns' in request URL");
|
||||
return;
|
||||
}
|
||||
|
||||
std::string query = params.get("query");
|
||||
std::string columns = params.get("columns");
|
||||
|
||||
auto names_and_types = parseColumns(std::move(columns), log);
|
||||
|
||||
if (!names_and_types)
|
||||
if (!params.has("connection_string"))
|
||||
{
|
||||
process_error("ODBCBridge: Invalid 'columns' parameter in request body");
|
||||
process_error("ODBCBridge: No 'connection_string' in request URL");
|
||||
return;
|
||||
}
|
||||
|
||||
Block sample_block;
|
||||
for (const NameAndTypePair & column_data : *names_and_types)
|
||||
sample_block.insert({column_data.type, column_data.name});
|
||||
|
||||
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);
|
||||
|
||||
std::shared_ptr<IBlockOutputStream> writer = FormatFactory::instance().getOutput(format, out, sample_block, *context);
|
||||
size_t max_block_size = parseMaxBlockSize(params.get("max_block_size", ""), log);
|
||||
|
||||
std::string columns = params.get("columns");
|
||||
std::unique_ptr<Block> sample_block;
|
||||
try
|
||||
{
|
||||
ODBCBlockInputStream inp(pool->get(), query, sample_block, max_block_size);
|
||||
sample_block = parseColumns(std::move(columns));
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
process_error("ODBCBridge: Invalid 'columns' parameter in request body '" + ex.message() + "'");
|
||||
return;
|
||||
}
|
||||
|
||||
writer->writePrefix();
|
||||
while (auto block = inp.read())
|
||||
writer->write(block);
|
||||
std::string format = params.get("format", "RowBinary");
|
||||
std::string query = params.get("query");
|
||||
LOG_TRACE(log, "ODBCBridge: Query '" << query << "'");
|
||||
|
||||
writer->writeSuffix();
|
||||
std::string connection_string = params.get("connection_string");
|
||||
LOG_TRACE(log, "ODBCBridge: Connection string '" << connection_string << "'");
|
||||
|
||||
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);
|
||||
try
|
||||
{
|
||||
|
||||
writer->flush();
|
||||
BlockOutputStreamPtr writer = FormatFactory::instance().getOutput(format, out, *sample_block, *context);
|
||||
auto pool = getPool(connection_string);
|
||||
ODBCBlockInputStream inp(pool->get(), query, *sample_block, max_block_size);
|
||||
copyData(inp, *writer);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
auto message = "Communication with ODBC-driver failed: \n" + getCurrentExceptionMessage(true);
|
||||
auto message = "ODBCBridge:\n" + getCurrentExceptionMessage(true);
|
||||
response.setStatusAndReason(
|
||||
Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); // can't call process_error, bacause of too soon response sending
|
||||
writeStringBinary(message, out);
|
||||
@ -123,7 +137,6 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void PingHandler::handleRequest(Poco::Net::HTTPServerRequest & /*request*/, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
try
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
namespace DB
|
||||
@ -14,15 +14,14 @@ namespace DB
|
||||
class ODBCHandler : public Poco::Net::HTTPRequestHandler
|
||||
{
|
||||
public:
|
||||
ODBCHandler(std::shared_ptr<Poco::Data::SessionPool> pool_,
|
||||
const std::string & format_,
|
||||
size_t max_block_size_,
|
||||
using PoolPtr = std::shared_ptr<Poco::Data::SessionPool>;
|
||||
using PoolMap = std::unordered_map<std::string, PoolPtr>;
|
||||
|
||||
ODBCHandler(std::shared_ptr<PoolMap> pool_map_,
|
||||
size_t keep_alive_timeout_,
|
||||
std::shared_ptr<Context> context_)
|
||||
: log(&Poco::Logger::get("ODBCHandler"))
|
||||
, pool(pool_)
|
||||
, format(format_)
|
||||
, max_block_size(max_block_size_)
|
||||
, pool_map(pool_map_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, context(context_)
|
||||
{
|
||||
@ -32,11 +31,14 @@ public:
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
std::shared_ptr<Poco::Data::SessionPool> pool;
|
||||
std::string format;
|
||||
size_t max_block_size;
|
||||
|
||||
std::shared_ptr<PoolMap> pool_map;
|
||||
size_t keep_alive_timeout;
|
||||
std::shared_ptr<Context> context;
|
||||
|
||||
static inline std::mutex mutex;
|
||||
|
||||
PoolPtr getPool(const std::string & connection_str);
|
||||
};
|
||||
|
||||
class PingHandler : public Poco::Net::HTTPRequestHandler
|
||||
|
@ -126,7 +126,7 @@ void ODBCBridge::initialize(Application & self)
|
||||
log = &logger();
|
||||
hostname = config().getString("http-host", "localhost");
|
||||
port = config().getUInt("http-port");
|
||||
if (port < 0 || port > 0xFFFF)
|
||||
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);
|
||||
|
Loading…
Reference in New Issue
Block a user