mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
wip
This commit is contained in:
parent
d055d82777
commit
63eaa97086
@ -139,7 +139,7 @@ public:
|
|||||||
|
|
||||||
const auto name = config.getString(config_prefix + ".executable.name");
|
const auto name = config.getString(config_prefix + ".executable.name");
|
||||||
const auto format = config.getString(config_prefix + ".executable.format");
|
const auto format = config.getString(config_prefix + ".executable.format");
|
||||||
return std::make_unique<ExecutableDictionarySource>(name, format, sample_block, context);
|
return std::make_unique<ExecutableDictionarySource>(dict_struct, name, format, sample_block, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
else if ("http" == source_type)
|
else if ("http" == source_type)
|
||||||
@ -149,7 +149,7 @@ public:
|
|||||||
"Dictionary source of type `http` does not support attribute expressions",
|
"Dictionary source of type `http` does not support attribute expressions",
|
||||||
ErrorCodes::LOGICAL_ERROR};
|
ErrorCodes::LOGICAL_ERROR};
|
||||||
|
|
||||||
return std::make_unique<HTTPDictionarySource>(config, config_prefix + ".http", sample_block, context);
|
return std::make_unique<HTTPDictionarySource>(dict_struct, config, config_prefix + ".http", sample_block, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
throw Exception{
|
throw Exception{
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Dictionaries/IDictionarySource.h>
|
#include <DB/Dictionaries/IDictionarySource.h>
|
||||||
|
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -13,7 +14,7 @@ class ExecutableDictionarySource final : public IDictionarySource
|
|||||||
|
|
||||||
public:
|
public:
|
||||||
|
|
||||||
ExecutableDictionarySource(const std::string & name, const std::string & format, Block & sample_block,
|
ExecutableDictionarySource(const DictionaryStructure & dict_struct_, const std::string & name, const std::string & format, Block & sample_block,
|
||||||
const Context & context);
|
const Context & context);
|
||||||
|
|
||||||
ExecutableDictionarySource(const ExecutableDictionarySource & other);
|
ExecutableDictionarySource(const ExecutableDictionarySource & other);
|
||||||
@ -36,13 +37,11 @@ public:
|
|||||||
private:
|
private:
|
||||||
Logger * log = &Logger::get("ExecutableDictionarySource");
|
Logger * log = &Logger::get("ExecutableDictionarySource");
|
||||||
|
|
||||||
LocalDateTime getLastModification() const;
|
const DictionaryStructure dict_struct;
|
||||||
|
|
||||||
const std::string name;
|
const std::string name;
|
||||||
const std::string format;
|
const std::string format;
|
||||||
Block sample_block;
|
Block sample_block;
|
||||||
const Context & context;
|
const Context & context;
|
||||||
LocalDateTime last_modification;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Dictionaries/IDictionarySource.h>
|
#include <DB/Dictionaries/IDictionarySource.h>
|
||||||
|
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -13,7 +14,7 @@ class HTTPDictionarySource final : public IDictionarySource
|
|||||||
|
|
||||||
public:
|
public:
|
||||||
|
|
||||||
HTTPDictionarySource(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block,
|
HTTPDictionarySource(const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block,
|
||||||
const Context & context);
|
const Context & context);
|
||||||
|
|
||||||
HTTPDictionarySource(const HTTPDictionarySource & other);
|
HTTPDictionarySource(const HTTPDictionarySource & other);
|
||||||
@ -41,6 +42,7 @@ private:
|
|||||||
|
|
||||||
LocalDateTime getLastModification() const;
|
LocalDateTime getLastModification() const;
|
||||||
|
|
||||||
|
const DictionaryStructure dict_struct;
|
||||||
const std::string host;
|
const std::string host;
|
||||||
int port;
|
int port;
|
||||||
const std::string path;
|
const std::string path;
|
||||||
@ -50,7 +52,6 @@ private:
|
|||||||
Block sample_block;
|
Block sample_block;
|
||||||
const Context & context;
|
const Context & context;
|
||||||
const std::string load_all_query;
|
const std::string load_all_query;
|
||||||
LocalDateTime last_modification;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -4,26 +4,34 @@
|
|||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/Dictionaries/OwningBlockInputStream.h>
|
#include <DB/Dictionaries/OwningBlockInputStream.h>
|
||||||
|
|
||||||
|
//#include <DB/IO/WriteBufferFromOStream.h>
|
||||||
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
ExecutableDictionarySource::ExecutableDictionarySource(const std::string & name, const std::string & format, Block & sample_block, const Context & context)
|
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_, const std::string & name, const std::string & format, Block & sample_block, const Context & context) :
|
||||||
: name{name}, format{format}, sample_block{sample_block}, context(context)
|
dict_struct{dict_struct_},
|
||||||
|
name{name},
|
||||||
|
format{format},
|
||||||
|
sample_block{sample_block},
|
||||||
|
context(context)
|
||||||
{
|
{
|
||||||
last_modification = std::time(nullptr);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other)
|
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other) :
|
||||||
: name{other.name},
|
dict_struct{other.dict_struct},
|
||||||
|
name{other.name},
|
||||||
format{other.format},
|
format{other.format},
|
||||||
sample_block{other.sample_block}, context(other.context),
|
sample_block{other.sample_block},
|
||||||
last_modification{other.last_modification}
|
context(other.context)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
|
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
|
||||||
{
|
{
|
||||||
last_modification = getLastModification();
|
//std::cerr << "ExecutableDictionarySource::loadAll " <<std::endl;
|
||||||
LOG_TRACE(log, "execute " + name);
|
LOG_TRACE(log, "execute " + name);
|
||||||
auto process = ShellCommand::execute(name);
|
auto process = ShellCommand::execute(name);
|
||||||
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
||||||
@ -32,20 +40,39 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll()
|
|||||||
|
|
||||||
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||||
{
|
{
|
||||||
|
//std::cerr << "ExecutableDictionarySource::loadIds s=" << ids.size() <<std::endl;
|
||||||
auto process = ShellCommand::execute(name);
|
auto process = ShellCommand::execute(name);
|
||||||
|
|
||||||
|
{
|
||||||
|
ColumnWithTypeAndName column;
|
||||||
|
column.type = std::make_shared<DataTypeUInt64>();
|
||||||
|
column.column = column.type->createColumn();
|
||||||
|
|
||||||
|
for (auto & id : ids) {
|
||||||
|
column.column->insert(id); //maybe faster?
|
||||||
|
}
|
||||||
|
|
||||||
|
Block block;
|
||||||
|
block.insert(std::move(column));
|
||||||
|
|
||||||
|
auto stream_out = context.getOutputFormat(format, process->in, sample_block);
|
||||||
|
stream_out->write(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
for (auto & id : ids) {
|
for (auto & id : ids) {
|
||||||
writeString(std::to_string(id), process->in);
|
writeString(std::to_string(id), process->in);
|
||||||
writeString("\n", process->in); // TODO: format?
|
writeString("\n", process->in);
|
||||||
}
|
}
|
||||||
|
*/
|
||||||
|
|
||||||
process->in.close();
|
process->in.close();
|
||||||
|
|
||||||
/*
|
/*
|
||||||
std::string process_err;
|
std::string process_err;
|
||||||
readStringUntilEOF(process_err, process->err);
|
readStringUntilEOF(process_err, process->err);
|
||||||
std::cerr << "readed ERR [" << process_err << "] " << std::endl;
|
std::cerr << "readed ERR [" << process_err << "] " << std::endl;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
auto stream = context.getInputFormat( format, process->out, sample_block, max_block_size);
|
auto stream = context.getInputFormat( format, process->out, sample_block, max_block_size);
|
||||||
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
||||||
@ -54,13 +81,48 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64
|
|||||||
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
|
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
|
||||||
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
|
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
|
||||||
{
|
{
|
||||||
std::cerr << "ExecutableDictionarySource::loadKeys " << requested_rows.size() << std::endl;
|
//std::cerr << " ExecutableDictionarySource::loadKeys cols=" << key_columns.size() << " rows=" <<requested_rows.size() << std::endl;
|
||||||
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
|
auto process = ShellCommand::execute(name);
|
||||||
|
|
||||||
|
{
|
||||||
|
Block block;
|
||||||
|
for(auto & key : key_columns) {
|
||||||
|
ColumnWithTypeAndName column;
|
||||||
|
column.type = std::make_shared<DataTypeUInt64>(); // TODO TYPE
|
||||||
|
//column.column = column.type->createColumn();
|
||||||
|
//column.column.reset(const_cast<DB::IColumn*>(key));
|
||||||
|
column.column = key->clone(); // wrong!
|
||||||
|
//column.column = key->convertToFullColumnIfConst(); // check!
|
||||||
|
block.insert(std::move(column));
|
||||||
|
}
|
||||||
|
|
||||||
|
auto stream_out = context.getOutputFormat(format, process->in, sample_block);
|
||||||
|
stream_out->write(block);
|
||||||
|
|
||||||
|
}
|
||||||
|
/*
|
||||||
|
for (const auto row : requested_rows)
|
||||||
|
{
|
||||||
|
writeString(std::to_string(row), process->in);
|
||||||
|
writeString("\n", process->in); // TODO: format?
|
||||||
|
}
|
||||||
|
*/
|
||||||
|
|
||||||
|
process->in.close();
|
||||||
|
|
||||||
|
/*
|
||||||
|
std::string process_err;
|
||||||
|
readStringUntilEOF(process_err, process->err);
|
||||||
|
std::cerr << "readed ERR [" << process_err << "] " << std::endl;
|
||||||
|
*/
|
||||||
|
|
||||||
|
auto stream = context.getInputFormat( format, process->out, sample_block, max_block_size);
|
||||||
|
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ExecutableDictionarySource::isModified() const
|
bool ExecutableDictionarySource::isModified() const
|
||||||
{
|
{
|
||||||
return getLastModification() > last_modification;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ExecutableDictionarySource::supportsSelectiveLoad() const
|
bool ExecutableDictionarySource::supportsSelectiveLoad() const
|
||||||
@ -78,10 +140,4 @@ std::string ExecutableDictionarySource::toString() const
|
|||||||
return "Executable: " + name;
|
return "Executable: " + name;
|
||||||
}
|
}
|
||||||
|
|
||||||
LocalDateTime ExecutableDictionarySource::getLastModification() const
|
|
||||||
{
|
|
||||||
return last_modification;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -8,7 +8,8 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
HTTPDictionarySource::HTTPDictionarySource(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")},
|
host{config.getString(config_prefix + ".host")},
|
||||||
port{std::stoi(config.getString(config_prefix + ".port"))},
|
port{std::stoi(config.getString(config_prefix + ".port"))},
|
||||||
path{config.getString(config_prefix + ".path")},
|
path{config.getString(config_prefix + ".path")},
|
||||||
@ -17,24 +18,23 @@ HTTPDictionarySource::HTTPDictionarySource(const Poco::Util::AbstractConfigurati
|
|||||||
sample_block{sample_block},
|
sample_block{sample_block},
|
||||||
context(context)
|
context(context)
|
||||||
{
|
{
|
||||||
last_modification = std::time(nullptr);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) :
|
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) :
|
||||||
|
dict_struct{other.dict_struct},
|
||||||
host{other.host},
|
host{other.host},
|
||||||
port{other.port},
|
port{other.port},
|
||||||
path{other.path},
|
path{other.path},
|
||||||
format{other.format},
|
format{other.format},
|
||||||
sample_block{other.sample_block}, context(other.context),
|
sample_block{other.sample_block},
|
||||||
last_modification{other.last_modification}
|
context(other.context)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreamPtr HTTPDictionarySource::loadAll()
|
BlockInputStreamPtr HTTPDictionarySource::loadAll()
|
||||||
{
|
{
|
||||||
auto in_ptr = std::make_unique<ReadBufferFromHTTP>(host, port, path, ReadBufferFromHTTP::Params(), method);
|
auto in_ptr = std::make_unique<ReadBufferFromHTTP>(host, port, path, ReadBufferFromHTTP::Params(), method);
|
||||||
auto stream = context.getInputFormat( format, *in_ptr, sample_block, max_block_size);
|
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
|
||||||
return std::make_shared<OwningBufferBlockInputStream>(stream, std::move(in_ptr));
|
return std::make_shared<OwningBufferBlockInputStream>(stream, std::move(in_ptr));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -51,7 +51,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(
|
|||||||
|
|
||||||
bool HTTPDictionarySource::isModified() const
|
bool HTTPDictionarySource::isModified() const
|
||||||
{
|
{
|
||||||
return getLastModification() > last_modification;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool HTTPDictionarySource::supportsSelectiveLoad() const
|
bool HTTPDictionarySource::supportsSelectiveLoad() const
|
||||||
@ -69,9 +69,4 @@ std::string HTTPDictionarySource::toString() const
|
|||||||
return "http://" + host + ":" + std::to_string(port) + "/" + path;
|
return "http://" + host + ":" + std::to_string(port) + "/" + path;
|
||||||
}
|
}
|
||||||
|
|
||||||
LocalDateTime HTTPDictionarySource::getLastModification() const
|
|
||||||
{
|
|
||||||
return last_modification;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user