mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 12:14:18 +00:00
Factorize generic code from ExternalDictLibBridgeHelper into base class
- When catboost will be added into library-bridge later on, a lot of code will need to be duplicated. - Avoid that by factorizing stuff which is not specific to external dictionaries for reuse into a common base class. - This is a refactoring without semantic change.
This commit is contained in:
parent
8fe8208133
commit
e584d1276b
@ -31,16 +31,10 @@ ExternalDictionaryLibraryBridgeHelper::ExternalDictionaryLibraryBridgeHelper(
|
||||
const Block & sample_block_,
|
||||
const Field & dictionary_id_,
|
||||
const LibraryInitData & library_data_)
|
||||
: IBridgeHelper(context_->getGlobalContext())
|
||||
, log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeHelper"))
|
||||
: LibraryBridgeHelper(context_->getGlobalContext())
|
||||
, sample_block(sample_block_)
|
||||
, config(context_->getConfigRef())
|
||||
, http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value)
|
||||
, library_data(library_data_)
|
||||
, dictionary_id(dictionary_id_)
|
||||
, bridge_host(config.getString("library_bridge.host", DEFAULT_HOST))
|
||||
, bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT))
|
||||
, http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -71,22 +65,6 @@ Poco::URI ExternalDictionaryLibraryBridgeHelper::createRequestURI(const String &
|
||||
}
|
||||
|
||||
|
||||
Poco::URI ExternalDictionaryLibraryBridgeHelper::createBaseURI() const
|
||||
{
|
||||
Poco::URI uri;
|
||||
uri.setHost(bridge_host);
|
||||
uri.setPort(bridge_port);
|
||||
uri.setScheme("http");
|
||||
return uri;
|
||||
}
|
||||
|
||||
|
||||
void ExternalDictionaryLibraryBridgeHelper::startBridge(std::unique_ptr<ShellCommand> cmd) const
|
||||
{
|
||||
getContext()->addBridgeCommand(std::move(cmd));
|
||||
}
|
||||
|
||||
|
||||
bool ExternalDictionaryLibraryBridgeHelper::bridgeHandShake()
|
||||
{
|
||||
String result;
|
||||
@ -225,6 +203,14 @@ QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadAll()
|
||||
}
|
||||
|
||||
|
||||
static String getDictIdsString(const std::vector<UInt64> & ids)
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
writeVectorBinary(ids, out);
|
||||
return out.str();
|
||||
}
|
||||
|
||||
|
||||
QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadIds(const std::vector<uint64_t> & ids)
|
||||
{
|
||||
startBridgeSync();
|
||||
@ -283,13 +269,4 @@ QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadBase(const Poco::URI &
|
||||
return QueryPipeline(std::move(source));
|
||||
}
|
||||
|
||||
|
||||
String ExternalDictionaryLibraryBridgeHelper::getDictIdsString(const std::vector<UInt64> & ids)
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
writeVectorBinary(ids, out);
|
||||
return out.str();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -2,10 +2,9 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <BridgeHelper/IBridgeHelper.h>
|
||||
#include <BridgeHelper/LibraryBridgeHelper.h>
|
||||
#include <QueryPipeline/QueryPipeline.h>
|
||||
|
||||
|
||||
@ -14,7 +13,8 @@ namespace DB
|
||||
|
||||
class Pipe;
|
||||
|
||||
class ExternalDictionaryLibraryBridgeHelper : public IBridgeHelper
|
||||
// Class to access the external dictionary part of the clickhouse-library-bridge.
|
||||
class ExternalDictionaryLibraryBridgeHelper : public LibraryBridgeHelper
|
||||
{
|
||||
|
||||
public:
|
||||
@ -25,7 +25,6 @@ public:
|
||||
String dict_attributes;
|
||||
};
|
||||
|
||||
static constexpr inline size_t DEFAULT_PORT = 9012;
|
||||
static constexpr inline auto PING_HANDLER = "/extdict_ping";
|
||||
static constexpr inline auto MAIN_HANDLER = "/extdict_request";
|
||||
|
||||
@ -56,26 +55,6 @@ protected:
|
||||
|
||||
bool bridgeHandShake() override;
|
||||
|
||||
void startBridge(std::unique_ptr<ShellCommand> cmd) const override;
|
||||
|
||||
String serviceAlias() const override { return "clickhouse-library-bridge"; }
|
||||
|
||||
String serviceFileName() const override { return serviceAlias(); }
|
||||
|
||||
size_t getDefaultPort() const override { return DEFAULT_PORT; }
|
||||
|
||||
bool startBridgeManually() const override { return false; }
|
||||
|
||||
String configPrefix() const override { return "library_bridge"; }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||
|
||||
Poco::Logger * getLog() const override { return log; }
|
||||
|
||||
Poco::Timespan getHTTPTimeout() const override { return http_timeout; }
|
||||
|
||||
Poco::URI createBaseURI() const override;
|
||||
|
||||
QueryPipeline loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {});
|
||||
|
||||
bool executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}) const;
|
||||
@ -94,20 +73,10 @@ private:
|
||||
|
||||
Poco::URI createRequestURI(const String & method) const;
|
||||
|
||||
static String getDictIdsString(const std::vector<UInt64> & ids);
|
||||
|
||||
Poco::Logger * log;
|
||||
const Block sample_block;
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
const Poco::Timespan http_timeout;
|
||||
|
||||
LibraryInitData library_data;
|
||||
Field dictionary_id;
|
||||
std::string bridge_host;
|
||||
size_t bridge_port;
|
||||
bool library_initialized = false;
|
||||
ConnectionTimeouts http_timeouts;
|
||||
Poco::Net::HTTPBasicCredentials credentials{};
|
||||
};
|
||||
|
||||
}
|
||||
|
34
src/BridgeHelper/LibraryBridgeHelper.cpp
Normal file
34
src/BridgeHelper/LibraryBridgeHelper.cpp
Normal file
@ -0,0 +1,34 @@
|
||||
#include "LibraryBridgeHelper.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_)
|
||||
: IBridgeHelper(context_)
|
||||
, config(context_->getConfigRef())
|
||||
, log(&Poco::Logger::get("LibraryBridgeHelper"))
|
||||
, http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value)
|
||||
, bridge_host(config.getString("library_bridge.host", DEFAULT_HOST))
|
||||
, bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT))
|
||||
, http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void LibraryBridgeHelper::startBridge(std::unique_ptr<ShellCommand> cmd) const
|
||||
{
|
||||
getContext()->addBridgeCommand(std::move(cmd));
|
||||
}
|
||||
|
||||
|
||||
Poco::URI LibraryBridgeHelper::createBaseURI() const
|
||||
{
|
||||
Poco::URI uri;
|
||||
uri.setHost(bridge_host);
|
||||
uri.setPort(bridge_port);
|
||||
uri.setScheme("http");
|
||||
return uri;
|
||||
}
|
||||
|
||||
|
||||
}
|
50
src/BridgeHelper/LibraryBridgeHelper.h
Normal file
50
src/BridgeHelper/LibraryBridgeHelper.h
Normal file
@ -0,0 +1,50 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <BridgeHelper/IBridgeHelper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// Common base class to access the clickhouse-library-bridge.
|
||||
class LibraryBridgeHelper : public IBridgeHelper
|
||||
{
|
||||
protected:
|
||||
explicit LibraryBridgeHelper(ContextPtr context_);
|
||||
|
||||
void startBridge(std::unique_ptr<ShellCommand> cmd) const override;
|
||||
|
||||
String serviceAlias() const override { return "clickhouse-library-bridge"; }
|
||||
|
||||
String serviceFileName() const override { return serviceAlias(); }
|
||||
|
||||
size_t getDefaultPort() const override { return DEFAULT_PORT; }
|
||||
|
||||
bool startBridgeManually() const override { return false; }
|
||||
|
||||
String configPrefix() const override { return "library_bridge"; }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||
|
||||
Poco::Logger * getLog() const override { return log; }
|
||||
|
||||
Poco::Timespan getHTTPTimeout() const override { return http_timeout; }
|
||||
|
||||
Poco::URI createBaseURI() const override;
|
||||
|
||||
static constexpr inline size_t DEFAULT_PORT = 9012;
|
||||
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
Poco::Logger * log;
|
||||
const Poco::Timespan http_timeout;
|
||||
std::string bridge_host;
|
||||
size_t bridge_port;
|
||||
ConnectionTimeouts http_timeouts;
|
||||
Poco::Net::HTTPBasicCredentials credentials{};
|
||||
};
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user