2017-05-25 20:59:14 +00:00
|
|
|
#include <thread>
|
|
|
|
#include <future>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Dictionaries/ExecutableDictionarySource.h>
|
|
|
|
#include <Common/ShellCommand.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2017-05-25 19:26:17 +00:00
|
|
|
#include <DataStreams/OwningBlockInputStream.h>
|
2017-05-25 19:21:57 +00:00
|
|
|
#include <Dictionaries/DictionarySourceHelpers.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
|
|
|
|
|
2016-11-15 19:51:06 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-12-08 02:49:04 +00:00
|
|
|
static const size_t max_block_size = 8192;
|
|
|
|
|
|
|
|
|
2017-10-12 19:40:42 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2017-10-12 19:35:33 +00:00
|
|
|
/// Owns ShellCommand and calls wait for it.
|
2017-10-02 13:08:09 +00:00
|
|
|
class ShellCommandOwningBlockInputStream : public OwningBlockInputStream<ShellCommand>
|
|
|
|
{
|
|
|
|
public:
|
2017-10-03 13:04:16 +00:00
|
|
|
ShellCommandOwningBlockInputStream(const BlockInputStreamPtr & stream, std::unique_ptr<ShellCommand> own)
|
|
|
|
: OwningBlockInputStream(std::move(stream), std::move(own))
|
2017-10-02 13:08:09 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void readSuffix() override
|
|
|
|
{
|
|
|
|
OwningBlockInputStream<ShellCommand>::readSuffix();
|
|
|
|
own->wait();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-10-12 19:40:42 +00:00
|
|
|
}
|
|
|
|
|
2017-10-12 19:35:01 +00:00
|
|
|
|
2016-11-22 15:03:54 +00:00
|
|
|
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
|
|
|
Block & sample_block, const Context & context)
|
|
|
|
: log(&Logger::get("ExecutableDictionarySource")),
|
2018-02-15 13:08:23 +00:00
|
|
|
update_time{std::chrono::system_clock::from_time_t(0)},
|
2017-04-01 07:20:54 +00:00
|
|
|
dict_struct{dict_struct_},
|
|
|
|
command{config.getString(config_prefix + ".command")},
|
2018-01-15 12:44:39 +00:00
|
|
|
update_field{config.getString(config_prefix + ".update_field", "")},
|
2017-04-01 07:20:54 +00:00
|
|
|
format{config.getString(config_prefix + ".format")},
|
|
|
|
sample_block{sample_block},
|
|
|
|
context(context)
|
2016-11-15 19:51:06 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2017-01-21 04:24:28 +00:00
|
|
|
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other)
|
2017-04-01 07:20:54 +00:00
|
|
|
: log(&Logger::get("ExecutableDictionarySource")),
|
2018-01-15 12:44:39 +00:00
|
|
|
update_time{other.update_time},
|
2017-04-01 07:20:54 +00:00
|
|
|
dict_struct{other.dict_struct},
|
|
|
|
command{other.command},
|
2018-02-15 13:08:23 +00:00
|
|
|
update_field{other.update_field},
|
2017-04-01 07:20:54 +00:00
|
|
|
format{other.format},
|
|
|
|
sample_block{other.sample_block},
|
|
|
|
context(other.context)
|
2016-11-15 19:51:06 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2018-02-15 13:08:23 +00:00
|
|
|
std::string ExecutableDictionarySource::getUpdateFieldAndDate()
|
2018-01-15 12:44:39 +00:00
|
|
|
{
|
2018-02-15 13:08:23 +00:00
|
|
|
if (update_time != std::chrono::system_clock::from_time_t(0))
|
|
|
|
{
|
2018-01-15 12:44:39 +00:00
|
|
|
auto tmp_time = update_time;
|
|
|
|
update_time = std::chrono::system_clock::now();
|
2018-02-15 13:08:23 +00:00
|
|
|
time_t hr_time = std::chrono::system_clock::to_time_t(tmp_time) - 1;
|
2018-01-15 12:44:39 +00:00
|
|
|
char buffer [80];
|
|
|
|
struct tm * timeinfo;
|
|
|
|
timeinfo = localtime (&hr_time);
|
2018-02-15 13:08:23 +00:00
|
|
|
strftime(buffer, 80, "\"%Y-%m-%d %H:%M:%S\"", timeinfo);
|
2018-01-15 12:44:39 +00:00
|
|
|
std::string str_time(buffer);
|
2018-02-15 13:08:23 +00:00
|
|
|
return command + " " + update_field + " "+ str_time;
|
|
|
|
///Example case: command -T "2018-02-12 12:44:04"
|
|
|
|
///should return all entries after mentioned date
|
|
|
|
///if executable is eligible to return entries according to date.
|
|
|
|
///Where "-T" is passed as update_field.
|
2018-01-15 12:44:39 +00:00
|
|
|
}
|
2018-02-15 13:08:23 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
std::string str_time("\"0000-00-00 00:00:00\""); ///for initial load
|
|
|
|
return command + " " + update_field + " "+ str_time;
|
2018-01-15 12:44:39 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-11-15 19:51:06 +00:00
|
|
|
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_TRACE(log, "loadAll " + toString());
|
|
|
|
auto process = ShellCommand::execute(command);
|
2017-05-25 19:21:57 +00:00
|
|
|
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
2017-10-02 13:08:09 +00:00
|
|
|
return std::make_shared<ShellCommandOwningBlockInputStream>(input_stream, std::move(process));
|
2016-11-27 13:01:35 +00:00
|
|
|
}
|
|
|
|
|
2018-02-15 13:08:23 +00:00
|
|
|
BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
|
|
|
|
{
|
|
|
|
std::string command_update = getUpdateFieldAndDate();
|
|
|
|
LOG_TRACE(log, "loadUpdatedAll " + command_update);
|
|
|
|
auto process = ShellCommand::execute(command_update);
|
|
|
|
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
|
|
|
return std::make_shared<ShellCommandOwningBlockInputStream>(input_stream, std::move(process));
|
|
|
|
}
|
2017-05-25 20:59:14 +00:00
|
|
|
|
2017-10-12 19:40:42 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2017-05-25 20:59:14 +00:00
|
|
|
/** A stream, that also runs and waits for background thread
|
|
|
|
* (that will feed data into pipe to be read from the other side of the pipe).
|
|
|
|
*/
|
2017-09-08 03:47:27 +00:00
|
|
|
class BlockInputStreamWithBackgroundThread final : public IProfilingBlockInputStream
|
2017-05-25 20:59:14 +00:00
|
|
|
{
|
|
|
|
public:
|
|
|
|
BlockInputStreamWithBackgroundThread(
|
|
|
|
const BlockInputStreamPtr & stream_, std::unique_ptr<ShellCommand> && command_,
|
|
|
|
std::packaged_task<void()> && task_)
|
|
|
|
: stream{stream_}, command{std::move(command_)}, task(std::move(task_)),
|
|
|
|
thread([this]{ task(); command->in.close(); })
|
|
|
|
{
|
|
|
|
children.push_back(stream);
|
|
|
|
}
|
|
|
|
|
|
|
|
~BlockInputStreamWithBackgroundThread() override
|
|
|
|
{
|
|
|
|
if (thread.joinable())
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
readSuffix();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-03 20:39:06 +00:00
|
|
|
Block getHeader() const override { return stream->getHeader(); }
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2017-05-25 20:59:14 +00:00
|
|
|
private:
|
|
|
|
Block readImpl() override { return stream->read(); }
|
|
|
|
|
|
|
|
void readSuffix() override
|
|
|
|
{
|
2017-10-02 13:08:09 +00:00
|
|
|
IProfilingBlockInputStream::readSuffix();
|
|
|
|
if (!wait_called)
|
|
|
|
{
|
|
|
|
wait_called = true;
|
|
|
|
command->wait();
|
|
|
|
}
|
2017-05-25 20:59:14 +00:00
|
|
|
thread.join();
|
|
|
|
/// To rethrow an exception, if any.
|
|
|
|
task.get_future().get();
|
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override { return "WithBackgroundThread"; }
|
|
|
|
|
|
|
|
BlockInputStreamPtr stream;
|
|
|
|
std::unique_ptr<ShellCommand> command;
|
|
|
|
std::packaged_task<void()> task;
|
|
|
|
std::thread thread;
|
2017-10-02 13:08:09 +00:00
|
|
|
bool wait_called = false;
|
2017-05-25 20:59:14 +00:00
|
|
|
};
|
|
|
|
|
2017-10-12 19:40:42 +00:00
|
|
|
}
|
|
|
|
|
2017-05-25 20:59:14 +00:00
|
|
|
|
2016-11-27 13:01:35 +00:00
|
|
|
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
|
|
|
{
|
2017-05-25 18:24:47 +00:00
|
|
|
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
|
2017-04-01 07:20:54 +00:00
|
|
|
auto process = ShellCommand::execute(command);
|
2017-05-25 18:24:47 +00:00
|
|
|
|
2017-05-25 19:21:57 +00:00
|
|
|
auto output_stream = context.getOutputFormat(format, process->in, sample_block);
|
|
|
|
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
2017-05-25 20:59:14 +00:00
|
|
|
|
|
|
|
return std::make_shared<BlockInputStreamWithBackgroundThread>(
|
|
|
|
input_stream, std::move(process), std::packaged_task<void()>(
|
2017-07-24 05:18:00 +00:00
|
|
|
[output_stream, &ids]() mutable
|
2017-05-25 20:59:14 +00:00
|
|
|
{
|
|
|
|
formatIDs(output_stream, ids);
|
|
|
|
}));
|
2016-11-15 19:51:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
|
2017-07-21 06:35:58 +00:00
|
|
|
const Columns & key_columns, const std::vector<size_t> & requested_rows)
|
2016-11-15 19:51:06 +00:00
|
|
|
{
|
2017-05-25 18:24:47 +00:00
|
|
|
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
|
2017-04-01 07:20:54 +00:00
|
|
|
auto process = ShellCommand::execute(command);
|
2016-11-18 01:48:13 +00:00
|
|
|
|
2017-05-25 19:21:57 +00:00
|
|
|
auto output_stream = context.getOutputFormat(format, process->in, sample_block);
|
|
|
|
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
2017-05-25 20:59:14 +00:00
|
|
|
|
|
|
|
return std::make_shared<BlockInputStreamWithBackgroundThread>(
|
|
|
|
input_stream, std::move(process), std::packaged_task<void()>(
|
|
|
|
[output_stream, key_columns, &requested_rows, this]() mutable
|
|
|
|
{
|
|
|
|
formatKeys(dict_struct, output_stream, key_columns, requested_rows);
|
|
|
|
}));
|
2016-11-15 19:51:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool ExecutableDictionarySource::isModified() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2016-11-15 19:51:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool ExecutableDictionarySource::supportsSelectiveLoad() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2016-11-15 19:51:06 +00:00
|
|
|
}
|
|
|
|
|
2018-01-15 12:44:39 +00:00
|
|
|
bool ExecutableDictionarySource::hasUpdateField() const
|
|
|
|
{
|
|
|
|
if(update_field.empty())
|
|
|
|
return false;
|
|
|
|
else
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2016-11-15 19:51:06 +00:00
|
|
|
DictionarySourcePtr ExecutableDictionarySource::clone() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return std::make_unique<ExecutableDictionarySource>(*this);
|
2016-11-15 19:51:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
std::string ExecutableDictionarySource::toString() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return "Executable: " + command;
|
2016-11-15 19:51:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|