mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Everything [maybe] works!
This commit is contained in:
parent
0b40a13ea7
commit
e280569453
@ -137,9 +137,7 @@ public:
|
||||
"Dictionary source of type `executable` does not support attribute expressions",
|
||||
ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
const auto name = config.getString(config_prefix + ".executable.name");
|
||||
const auto format = config.getString(config_prefix + ".executable.format");
|
||||
return std::make_unique<ExecutableDictionarySource>(dict_struct, name, format, sample_block, context);
|
||||
return std::make_unique<ExecutableDictionarySource>(dict_struct, config, config_prefix + ".executable", sample_block, context);
|
||||
}
|
||||
|
||||
else if ("http" == source_type)
|
||||
|
@ -14,7 +14,9 @@ class ExecutableDictionarySource final : public IDictionarySource
|
||||
|
||||
public:
|
||||
|
||||
ExecutableDictionarySource(const DictionaryStructure & dict_struct_, const std::string & name, const std::string & format, Block & sample_block,
|
||||
ExecutableDictionarySource(const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context);
|
||||
|
||||
ExecutableDictionarySource(const ExecutableDictionarySource & other);
|
||||
@ -40,6 +42,7 @@ private:
|
||||
const DictionaryStructure dict_struct;
|
||||
const std::string name;
|
||||
const std::string format;
|
||||
const bool selective;
|
||||
Block sample_block;
|
||||
const Context & context;
|
||||
};
|
||||
|
@ -33,8 +33,6 @@ public:
|
||||
|
||||
DictionarySourcePtr clone() const override;
|
||||
|
||||
//DictionarySourcePtr clone() const override { return std::make_unique<HTTPDictionarySource>(*this); }
|
||||
|
||||
std::string toString() const override;
|
||||
|
||||
private:
|
||||
@ -43,12 +41,14 @@ private:
|
||||
LocalDateTime getLastModification() const;
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
|
||||
const std::string host;
|
||||
int port;
|
||||
const std::string path;
|
||||
//const std::string method;
|
||||
|
||||
const std::string format;
|
||||
const bool selective;
|
||||
|
||||
Block sample_block;
|
||||
const Context & context;
|
||||
const std::string load_all_query;
|
||||
|
@ -1,17 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
//#include <memory>
|
||||
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <DB/IO/ReadBufferFromHTTP.h>
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
//#include <DB/Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
struct HTTPLocation {
|
||||
using Params = std::vector<std::pair<String, String>>;
|
||||
std::string protocol = "http";
|
||||
@ -44,10 +40,12 @@ private:
|
||||
std::unique_ptr<ReadBuffer> impl;
|
||||
|
||||
public:
|
||||
using OutStreamCallback = std::function<void(std::ostream&)>;
|
||||
//using Params = std::vector<std::pair<String, String>>;
|
||||
|
||||
ReadWriteBufferFromHTTP(
|
||||
HTTPLocation http_query,
|
||||
OutStreamCallback out_stream_callback = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
HTTPTimeouts timeouts = HTTPTimeouts()
|
||||
);
|
||||
|
@ -4,17 +4,19 @@
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Dictionaries/OwningBlockInputStream.h>
|
||||
|
||||
//#include <DB/IO/WriteBufferFromOStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_, const std::string & name, const std::string & format, Block & sample_block, const Context & context) :
|
||||
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||
Block & sample_block, const Context & context) :
|
||||
dict_struct{dict_struct_},
|
||||
name{name},
|
||||
format{format},
|
||||
name{config.getString(config_prefix + ".name")},
|
||||
format{config.getString(config_prefix + ".format")},
|
||||
selective{!config.getString(config_prefix + ".selective", "").empty()}, // todo! how to correct?
|
||||
sample_block{sample_block},
|
||||
context(context)
|
||||
{
|
||||
@ -24,6 +26,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar
|
||||
dict_struct{other.dict_struct},
|
||||
name{other.name},
|
||||
format{other.format},
|
||||
selective{other.selective},
|
||||
sample_block{other.sample_block},
|
||||
context(other.context)
|
||||
{
|
||||
@ -116,7 +119,7 @@ bool ExecutableDictionarySource::isModified() const
|
||||
|
||||
bool ExecutableDictionarySource::supportsSelectiveLoad() const
|
||||
{
|
||||
return true;
|
||||
return selective;
|
||||
}
|
||||
|
||||
DictionarySourcePtr ExecutableDictionarySource::clone() const
|
||||
|
@ -6,16 +6,23 @@
|
||||
#include <DB/IO/ReadBufferFromHTTP.h>
|
||||
#include <DB/IO/ReadWriteBufferFromHTTP.h>
|
||||
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, const Context & context) :
|
||||
HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||
Block & sample_block, const Context & context) :
|
||||
dict_struct{dict_struct_},
|
||||
host{config.getString(config_prefix + ".host", "::1")},
|
||||
port{std::stoi(config.getString(config_prefix + ".port", "80"))},
|
||||
path{config.getString(config_prefix + ".path", "")},
|
||||
//method{config.getString(config_prefix + ".method", "")},
|
||||
format{config.getString(config_prefix + ".format")},
|
||||
selective{!config.getString(config_prefix + ".selective", "").empty()}, // todo! how to correct?
|
||||
sample_block{sample_block},
|
||||
context(context)
|
||||
{
|
||||
@ -27,6 +34,7 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) :
|
||||
port{other.port},
|
||||
path{other.path},
|
||||
format{other.format},
|
||||
selective{other.selective},
|
||||
sample_block{other.sample_block},
|
||||
context(other.context)
|
||||
{
|
||||
@ -43,14 +51,30 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll()
|
||||
BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||
{
|
||||
LOG_TRACE(log, "loadIds " + toString());
|
||||
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
||||
HTTPLocation http_location;
|
||||
http_location.host = host;
|
||||
http_location.port = port;
|
||||
http_location.path = path;
|
||||
//http_location.method = method;
|
||||
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(http_location);
|
||||
http_location.method = Poco::Net::HTTPRequest::HTTP_POST;
|
||||
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & out_stream) {
|
||||
// copypaste from ExecutableDictionarySource.cpp, todo: make func
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeUInt64>();
|
||||
column.column = column.type->createColumn();
|
||||
|
||||
for (auto & id : ids) {
|
||||
column.column->insert(id); //CHECKME maybe faster?
|
||||
}
|
||||
|
||||
Block block;
|
||||
block.insert(std::move(column));
|
||||
|
||||
WriteBufferFromOStream out_buffer(out_stream);
|
||||
auto stream_out = context.getOutputFormat(format, out_buffer, sample_block);
|
||||
stream_out->write(block);
|
||||
};
|
||||
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(http_location, out_stream_callback);
|
||||
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
|
||||
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(stream, std::move(in_ptr));
|
||||
|
||||
@ -60,7 +84,38 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(
|
||||
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
|
||||
{
|
||||
LOG_TRACE(log, "loadKeys " + toString());
|
||||
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
||||
HTTPLocation http_location;
|
||||
http_location.host = host;
|
||||
http_location.port = port;
|
||||
http_location.path = path;
|
||||
http_location.method = Poco::Net::HTTPRequest::HTTP_POST;
|
||||
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & out_stream) {
|
||||
// copypaste from ExecutableDictionarySource.cpp, todo: make func
|
||||
Block block;
|
||||
|
||||
const auto keys_size = key_columns.size();
|
||||
for (const auto i : ext::range(0, keys_size))
|
||||
{
|
||||
|
||||
const auto & key_description = (*dict_struct.key)[i];
|
||||
const auto & key = key_columns[i];
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = key_description.type;
|
||||
column.column = key->clone(); // CHECKME !!
|
||||
block.insert(std::move(column));
|
||||
}
|
||||
|
||||
WriteBufferFromOStream out_buffer(out_stream);
|
||||
auto stream_out = context.getOutputFormat(format, out_buffer, sample_block);
|
||||
stream_out->write(block);
|
||||
};
|
||||
|
||||
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(http_location, out_stream_callback);
|
||||
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
|
||||
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(stream, std::move(in_ptr));
|
||||
|
||||
}
|
||||
|
||||
bool HTTPDictionarySource::isModified() const
|
||||
@ -70,7 +125,7 @@ bool HTTPDictionarySource::isModified() const
|
||||
|
||||
bool HTTPDictionarySource::supportsSelectiveLoad() const
|
||||
{
|
||||
return true;
|
||||
return selective;
|
||||
}
|
||||
|
||||
DictionarySourcePtr HTTPDictionarySource::clone() const
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Poco/URI.h>
|
||||
#include <Poco/Net/DNS.h>
|
||||
//#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
@ -36,6 +35,7 @@ static Poco::Net::IPAddress resolveHost(const String & host)
|
||||
|
||||
ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
|
||||
HTTPLocation location,
|
||||
OutStreamCallback out_stream_callback,
|
||||
size_t buffer_size_,
|
||||
HTTPTimeouts timeouts
|
||||
) :
|
||||
@ -72,8 +72,12 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
|
||||
|
||||
LOG_TRACE((&Logger::get("ReadBufferFromHTTP")), "Sending request to " << uri.str());
|
||||
|
||||
//auto & stream_out =
|
||||
session.sendRequest(request);
|
||||
auto & stream_out = session.sendRequest(request);
|
||||
|
||||
if (out_stream_callback) {
|
||||
out_stream_callback(stream_out);
|
||||
}
|
||||
|
||||
istr = &session.receiveResponse(response);
|
||||
|
||||
auto status = response.getStatus();
|
||||
|
@ -17,7 +17,6 @@
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
#include <DB/IO/ReadBufferFromString.h>
|
||||
#include <DB/IO/Operators.h>
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user