mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Maybe
This commit is contained in:
parent
adbd2d65c2
commit
303813c1d8
100
src/DataStreams/ShellCommandBlockInputStream.h
Normal file
100
src/DataStreams/ShellCommandBlockInputStream.h
Normal file
@ -0,0 +1,100 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <memory>
|
||||||
|
|
||||||
|
#include <Common/ShellCommand.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <DataStreams/OwningBlockInputStream.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
/// Owns ShellCommand and calls wait for it.
|
||||||
|
class ShellCommandOwningBlockInputStream : public OwningBlockInputStream<ShellCommand>
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
Poco::Logger * log;
|
||||||
|
public:
|
||||||
|
ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr<ShellCommand> command_)
|
||||||
|
: OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
void readSuffix() override
|
||||||
|
{
|
||||||
|
OwningBlockInputStream<ShellCommand>::readSuffix();
|
||||||
|
|
||||||
|
std::string err;
|
||||||
|
readStringUntilEOF(err, own->err);
|
||||||
|
if (!err.empty())
|
||||||
|
LOG_ERROR(log, "Having stderr: {}", err);
|
||||||
|
|
||||||
|
own->wait();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
/** A stream, that runs child process and sends data to its stdin in background thread,
|
||||||
|
* and receives data from its stdout.
|
||||||
|
*/
|
||||||
|
class BlockInputStreamWithBackgroundThread final : public IBlockInputStream
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
BlockInputStreamWithBackgroundThread(
|
||||||
|
const Context & context,
|
||||||
|
const std::string & format,
|
||||||
|
const Block & sample_block,
|
||||||
|
const std::string & command_str,
|
||||||
|
Poco::Logger * log_,
|
||||||
|
UInt64 max_block_size,
|
||||||
|
std::function<void(WriteBufferFromFile &)> && send_data_)
|
||||||
|
: log(log_),
|
||||||
|
command(ShellCommand::execute(command_str)),
|
||||||
|
send_data(std::move(send_data_)),
|
||||||
|
thread([this] { send_data(command->in); })
|
||||||
|
{
|
||||||
|
stream = context.getInputFormat(format, command->out, sample_block, max_block_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
~BlockInputStreamWithBackgroundThread() override
|
||||||
|
{
|
||||||
|
if (thread.joinable())
|
||||||
|
thread.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
Block getHeader() const override
|
||||||
|
{
|
||||||
|
return stream->getHeader();
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
Block readImpl() override
|
||||||
|
{
|
||||||
|
return stream->read();
|
||||||
|
}
|
||||||
|
|
||||||
|
void readPrefix() override
|
||||||
|
{
|
||||||
|
stream->readPrefix();
|
||||||
|
}
|
||||||
|
|
||||||
|
void readSuffix() override
|
||||||
|
{
|
||||||
|
stream->readSuffix();
|
||||||
|
|
||||||
|
std::string err;
|
||||||
|
readStringUntilEOF(err, command->err);
|
||||||
|
if (!err.empty())
|
||||||
|
LOG_ERROR(log, "Having stderr: {}", err);
|
||||||
|
|
||||||
|
command->wait();
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return "WithBackgroundThread"; }
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
BlockInputStreamPtr stream;
|
||||||
|
std::unique_ptr<ShellCommand> command;
|
||||||
|
std::function<void(WriteBufferFromFile &)> send_data;
|
||||||
|
ThreadFromGlobalPool thread;
|
||||||
|
};
|
||||||
|
}
|
@ -1,37 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <memory>
|
|
||||||
|
|
||||||
#include <Common/ShellCommand.h>
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
#include <DataStreams/OwningBlockInputStream.h>
|
|
||||||
#include <IO/ReadHelpers.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
/// Owns ShellCommand and calls wait for it.
|
|
||||||
class ShellCommandOwningBlockInputStream : public OwningBlockInputStream<ShellCommand>
|
|
||||||
{
|
|
||||||
private:
|
|
||||||
Poco::Logger * log;
|
|
||||||
public:
|
|
||||||
ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr<ShellCommand> command_)
|
|
||||||
: OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
void readSuffix() override
|
|
||||||
{
|
|
||||||
OwningBlockInputStream<ShellCommand>::readSuffix();
|
|
||||||
|
|
||||||
std::string err;
|
|
||||||
readStringUntilEOF(err, own->err);
|
|
||||||
if (!err.empty())
|
|
||||||
LOG_ERROR(log, "Having stderr: {}", err);
|
|
||||||
|
|
||||||
own->wait();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -3,7 +3,7 @@
|
|||||||
#include <functional>
|
#include <functional>
|
||||||
#include <ext/scope_guard.h>
|
#include <ext/scope_guard.h>
|
||||||
#include <DataStreams/IBlockOutputStream.h>
|
#include <DataStreams/IBlockOutputStream.h>
|
||||||
#include <DataStreams/ShellCommandOwningBlockInputStream.h>
|
#include <DataStreams/ShellCommandBlockInputStream.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
@ -79,80 +79,12 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
|
|||||||
return std::make_shared<ShellCommandOwningBlockInputStream>(log, input_stream, std::move(process));
|
return std::make_shared<ShellCommandOwningBlockInputStream>(log, input_stream, std::move(process));
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
/** A stream, that runs child process and sends data to its stdin in background thread,
|
|
||||||
* and receives data from its stdout.
|
|
||||||
*/
|
|
||||||
class BlockInputStreamWithBackgroundThread final : public IBlockInputStream
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
BlockInputStreamWithBackgroundThread(
|
|
||||||
const Context & context,
|
|
||||||
const std::string & format,
|
|
||||||
const Block & sample_block,
|
|
||||||
const std::string & command_str,
|
|
||||||
Poco::Logger * log_,
|
|
||||||
std::function<void(WriteBufferFromFile &)> && send_data_)
|
|
||||||
: log(log_),
|
|
||||||
command(ShellCommand::execute(command_str)),
|
|
||||||
send_data(std::move(send_data_)),
|
|
||||||
thread([this] { send_data(command->in); })
|
|
||||||
{
|
|
||||||
stream = context.getInputFormat(format, command->out, sample_block, max_block_size);
|
|
||||||
}
|
|
||||||
|
|
||||||
~BlockInputStreamWithBackgroundThread() override
|
|
||||||
{
|
|
||||||
if (thread.joinable())
|
|
||||||
thread.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
Block getHeader() const override
|
|
||||||
{
|
|
||||||
return stream->getHeader();
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
Block readImpl() override
|
|
||||||
{
|
|
||||||
return stream->read();
|
|
||||||
}
|
|
||||||
|
|
||||||
void readPrefix() override
|
|
||||||
{
|
|
||||||
stream->readPrefix();
|
|
||||||
}
|
|
||||||
|
|
||||||
void readSuffix() override
|
|
||||||
{
|
|
||||||
stream->readSuffix();
|
|
||||||
|
|
||||||
std::string err;
|
|
||||||
readStringUntilEOF(err, command->err);
|
|
||||||
if (!err.empty())
|
|
||||||
LOG_ERROR(log, "Having stderr: {}", err);
|
|
||||||
|
|
||||||
command->wait();
|
|
||||||
}
|
|
||||||
|
|
||||||
String getName() const override { return "WithBackgroundThread"; }
|
|
||||||
|
|
||||||
Poco::Logger * log;
|
|
||||||
BlockInputStreamPtr stream;
|
|
||||||
std::unique_ptr<ShellCommand> command;
|
|
||||||
std::function<void(WriteBufferFromFile &)> send_data;
|
|
||||||
ThreadFromGlobalPool thread;
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
||||||
|
|
||||||
return std::make_shared<BlockInputStreamWithBackgroundThread>(
|
return std::make_shared<BlockInputStreamWithBackgroundThread>(
|
||||||
context, format, sample_block, command, log,
|
context, format, sample_block, command, log, max_block_size,
|
||||||
[&ids, this](WriteBufferFromFile & out) mutable
|
[&ids, this](WriteBufferFromFile & out) mutable
|
||||||
{
|
{
|
||||||
auto output_stream = context.getOutputFormat(format, out, sample_block);
|
auto output_stream = context.getOutputFormat(format, out, sample_block);
|
||||||
@ -166,7 +98,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_col
|
|||||||
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
||||||
|
|
||||||
return std::make_shared<BlockInputStreamWithBackgroundThread>(
|
return std::make_shared<BlockInputStreamWithBackgroundThread>(
|
||||||
context, format, sample_block, command, log,
|
context, format, sample_block, command, log, max_block_size,
|
||||||
[key_columns, &requested_rows, this](WriteBufferFromFile & out) mutable
|
[key_columns, &requested_rows, this](WriteBufferFromFile & out) mutable
|
||||||
{
|
{
|
||||||
auto output_stream = context.getOutputFormat(format, out, sample_block);
|
auto output_stream = context.getOutputFormat(format, out, sample_block);
|
||||||
|
@ -8,7 +8,8 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Storages/StorageFactory.h>
|
#include <Storages/StorageFactory.h>
|
||||||
#include <Processors/Sources/SourceWithProgress.h>
|
#include <Processors/Sources/SourceWithProgress.h>
|
||||||
#include <DataStreams/ShellCommandOwningBlockInputStream.h>
|
#include <DataStreams/ShellCommandBlockInputStream.h>
|
||||||
|
#include <DataStreams/IBlockOutputStream.h>
|
||||||
#include <Dictionaries/ExecutableDictionarySource.h>
|
#include <Dictionaries/ExecutableDictionarySource.h>
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
@ -31,18 +32,18 @@ public:
|
|||||||
String getName() const override { return "Executable"; }
|
String getName() const override { return "Executable"; }
|
||||||
|
|
||||||
StorageExecutableSource(
|
StorageExecutableSource(
|
||||||
|
const String & file_path_,
|
||||||
const String & format_,
|
const String & format_,
|
||||||
|
BlockInputStreamPtr input_,
|
||||||
const StorageMetadataPtr & metadata_snapshot_,
|
const StorageMetadataPtr & metadata_snapshot_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
const ColumnsDescription & /*columns*/,
|
UInt64 max_block_size_)
|
||||||
UInt64 max_block_size_,
|
|
||||||
const CompressionMethod /*compression_method*/,
|
|
||||||
const String & file_path_)
|
|
||||||
: SourceWithProgress(metadata_snapshot_->getSampleBlock())
|
: SourceWithProgress(metadata_snapshot_->getSampleBlock())
|
||||||
, file_path(std::move(file_path_))
|
, file_path(std::move(file_path_))
|
||||||
|
, format(format_)
|
||||||
|
, input(input_)
|
||||||
, metadata_snapshot(metadata_snapshot_)
|
, metadata_snapshot(metadata_snapshot_)
|
||||||
, context(context_)
|
, context(context_)
|
||||||
, format(format_)
|
|
||||||
, max_block_size(max_block_size_)
|
, max_block_size(max_block_size_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
@ -52,9 +53,29 @@ public:
|
|||||||
|
|
||||||
if (!reader)
|
if (!reader)
|
||||||
{
|
{
|
||||||
auto process = ShellCommand::execute(file_path);
|
auto sample_block = metadata_snapshot->getSampleBlock();
|
||||||
auto input_stream = context.getInputFormat(format, process->out, metadata_snapshot->getSampleBlock(), max_block_size);
|
reader = std::make_shared<BlockInputStreamWithBackgroundThread>(context, format, sample_block, file_path, log, max_block_size, [this](WriteBufferFromFile & out)
|
||||||
reader = std::make_shared<ShellCommandOwningBlockInputStream>(log, input_stream, std::move(process));
|
{
|
||||||
|
if (!input)
|
||||||
|
{
|
||||||
|
out.close();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto output_stream = context.getOutputFormat(format, out, input->getHeader());
|
||||||
|
output_stream->writePrefix();
|
||||||
|
|
||||||
|
input->readPrefix();
|
||||||
|
while (auto block = input->read()) {
|
||||||
|
output_stream->write(block);
|
||||||
|
}
|
||||||
|
input->readSuffix();
|
||||||
|
input.reset();
|
||||||
|
|
||||||
|
output_stream->writeSuffix();
|
||||||
|
output_stream->flush();
|
||||||
|
out.close();
|
||||||
|
});
|
||||||
|
|
||||||
reader->readPrefix();
|
reader->readPrefix();
|
||||||
}
|
}
|
||||||
@ -73,10 +94,11 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
String file_path;
|
String file_path;
|
||||||
|
const String & format;
|
||||||
|
BlockInputStreamPtr input;
|
||||||
BlockInputStreamPtr reader;
|
BlockInputStreamPtr reader;
|
||||||
const StorageMetadataPtr & metadata_snapshot;
|
const StorageMetadataPtr & metadata_snapshot;
|
||||||
const Context & context;
|
const Context & context;
|
||||||
const String & format;
|
|
||||||
UInt64 max_block_size;
|
UInt64 max_block_size;
|
||||||
Poco::Logger * log = &Poco::Logger::get("StorageExecutableSource");
|
Poco::Logger * log = &Poco::Logger::get("StorageExecutableSource");
|
||||||
};
|
};
|
||||||
@ -85,18 +107,17 @@ public:
|
|||||||
|
|
||||||
StorageExecutable::StorageExecutable(
|
StorageExecutable::StorageExecutable(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
const String & format_name_,
|
|
||||||
const String & file_path_,
|
const String & file_path_,
|
||||||
|
const String & format_,
|
||||||
|
BlockInputStreamPtr input_,
|
||||||
const ColumnsDescription & columns,
|
const ColumnsDescription & columns,
|
||||||
const ConstraintsDescription & constraints,
|
const ConstraintsDescription & constraints,
|
||||||
Context & context_,
|
const Context & context_)
|
||||||
CompressionMethod compression_method_)
|
|
||||||
: IStorage(table_id_)
|
: IStorage(table_id_)
|
||||||
, format_name(format_name_)
|
|
||||||
, file_path(file_path_)
|
, file_path(file_path_)
|
||||||
|
, format(format_)
|
||||||
|
, input(input_)
|
||||||
, context(context_)
|
, context(context_)
|
||||||
, compression_method(compression_method_)
|
|
||||||
, log(&Poco::Logger::get("StorageExecutable (" + table_id_.getFullTableName() + ")"))
|
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
storage_metadata.setColumns(columns);
|
storage_metadata.setColumns(columns);
|
||||||
@ -104,51 +125,6 @@ StorageExecutable::StorageExecutable(
|
|||||||
setInMemoryMetadata(storage_metadata);
|
setInMemoryMetadata(storage_metadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerStorageExecutable(StorageFactory & factory)
|
|
||||||
{
|
|
||||||
factory.registerStorage(
|
|
||||||
"Executable",
|
|
||||||
[](const StorageFactory::Arguments & args)
|
|
||||||
{
|
|
||||||
ASTs & engine_args = args.engine_args;
|
|
||||||
|
|
||||||
if (!(engine_args.size() >= 1 && engine_args.size() <= 4)) // NOLINT
|
|
||||||
throw Exception(
|
|
||||||
"Storage Executable requires from 1 to 4 arguments: name of used format, source and compression_method.",
|
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
|
|
||||||
String format_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
|
||||||
|
|
||||||
String compression_method_str;
|
|
||||||
String source_path;
|
|
||||||
|
|
||||||
if (const auto * literal = engine_args[1]->as<ASTLiteral>())
|
|
||||||
{
|
|
||||||
auto type = literal->value.getType();
|
|
||||||
if (type == Field::Types::String)
|
|
||||||
source_path = literal->value.get<String>();
|
|
||||||
else
|
|
||||||
throw Exception("Second argument must be path or file descriptor", ErrorCodes::BAD_ARGUMENTS);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (engine_args.size() == 3)
|
|
||||||
{
|
|
||||||
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
|
|
||||||
compression_method_str = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
|
||||||
}
|
|
||||||
|
|
||||||
CompressionMethod compression_method = chooseCompressionMethod("", compression_method_str);
|
|
||||||
|
|
||||||
return StorageExecutable::create(args.table_id, format_name, source_path, args.columns, args.constraints, args.context, compression_method);
|
|
||||||
},
|
|
||||||
{
|
|
||||||
.source_access_type = AccessType::FILE,
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
Pipe StorageExecutable::read(
|
Pipe StorageExecutable::read(
|
||||||
const Names & /*column_names*/,
|
const Names & /*column_names*/,
|
||||||
const StorageMetadataPtr & metadata_snapshot,
|
const StorageMetadataPtr & metadata_snapshot,
|
||||||
@ -159,13 +135,12 @@ Pipe StorageExecutable::read(
|
|||||||
unsigned /*num_streams*/)
|
unsigned /*num_streams*/)
|
||||||
{
|
{
|
||||||
return Pipe(std::make_shared<StorageExecutableSource>(
|
return Pipe(std::make_shared<StorageExecutableSource>(
|
||||||
format_name,
|
file_path,
|
||||||
|
format,
|
||||||
|
input,
|
||||||
metadata_snapshot,
|
metadata_snapshot,
|
||||||
context_,
|
context_,
|
||||||
metadata_snapshot->getColumns(),
|
max_block_size));
|
||||||
max_block_size,
|
|
||||||
compression_method,
|
|
||||||
file_path));
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -29,19 +29,18 @@ public:
|
|||||||
protected:
|
protected:
|
||||||
StorageExecutable(
|
StorageExecutable(
|
||||||
const StorageID & table_id,
|
const StorageID & table_id,
|
||||||
const String & format_name_,
|
|
||||||
const String & file_path_,
|
const String & file_path_,
|
||||||
|
const String & format_,
|
||||||
|
BlockInputStreamPtr input_,
|
||||||
const ColumnsDescription & columns,
|
const ColumnsDescription & columns,
|
||||||
const ConstraintsDescription & constraints,
|
const ConstraintsDescription & constraints,
|
||||||
Context & context_,
|
const Context & context_);
|
||||||
CompressionMethod compression_method_);
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
String format_name;
|
|
||||||
String file_path;
|
String file_path;
|
||||||
Context & context;
|
String format;
|
||||||
CompressionMethod compression_method;
|
BlockInputStreamPtr input;
|
||||||
Poco::Logger * log = &Poco::Logger::get("StorageExecutable");
|
const Context & context;
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -111,6 +111,7 @@ SRCS(
|
|||||||
StorageBuffer.cpp
|
StorageBuffer.cpp
|
||||||
StorageDictionary.cpp
|
StorageDictionary.cpp
|
||||||
StorageDistributed.cpp
|
StorageDistributed.cpp
|
||||||
|
StorageExecutable.cpp
|
||||||
StorageFactory.cpp
|
StorageFactory.cpp
|
||||||
StorageFile.cpp
|
StorageFile.cpp
|
||||||
StorageGenerateRandom.cpp
|
StorageGenerateRandom.cpp
|
||||||
|
@ -1,22 +1,71 @@
|
|||||||
#include <Storages/StorageExecutable.h>
|
#include <TableFunctions/ITableFunction.h>
|
||||||
#include <Storages/ColumnsDescription.h>
|
|
||||||
#include <Access/AccessFlags.h>
|
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
|
||||||
#include <TableFunctions/TableFunctionExecutable.h>
|
#include <TableFunctions/TableFunctionExecutable.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
#include <Interpreters/StorageID.h>
|
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
#include <Parsers/ASTSubquery.h>
|
||||||
|
#include <Common/Exception.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Storages/StorageExecutable.h>
|
||||||
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
|
#include <Interpreters/interpretSubquery.h>
|
||||||
|
#include <boost/algorithm/string.hpp>
|
||||||
#include "registerTableFunctions.h"
|
#include "registerTableFunctions.h"
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
StoragePtr TableFunctionExecutable::getStorage(
|
|
||||||
const String & source_path, const String & format_, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method_) const
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
return StorageExecutable::create(StorageID(getDatabaseName(), table_name), format_, source_path, columns, ConstraintsDescription{}, global_context, chooseCompressionMethod("", compression_method_));
|
extern const int LOGICAL_ERROR;
|
||||||
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
}
|
||||||
|
|
||||||
|
void TableFunctionExecutable::parseArguments(const ASTPtr & ast_function, const Context & context)
|
||||||
|
{
|
||||||
|
const auto * function = ast_function->as<ASTFunction>();
|
||||||
|
|
||||||
|
if (!function->arguments)
|
||||||
|
throw Exception("Table function '" + getName() + "' must have arguments", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto args = function->arguments->children;
|
||||||
|
|
||||||
|
if (!(args.size() == 3 || args.size() == 4))
|
||||||
|
throw Exception("Table function '" + getName() + "' requires exactly 3 or 4 arguments: path, format, structure, [input_query]",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
for (size_t i = 0; i <= 2; ++i)
|
||||||
|
args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context);
|
||||||
|
|
||||||
|
file_path = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context)->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
|
if (args.size() == 4) {
|
||||||
|
if (!(args[3]->as<ASTSubquery>()))
|
||||||
|
throw Exception("Table function '" + getName() + "' 4th argument is invalid input query", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
input = interpretSubquery(args[3], context, {}, {})->execute().getInputStream();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnsDescription TableFunctionExecutable::getActualTableStructure(const Context & context) const
|
||||||
|
{
|
||||||
|
return parseColumnsListFromString(structure, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
||||||
|
{
|
||||||
|
auto storage = StorageExecutable::create(StorageID(getDatabaseName(), table_name), file_path, format, input, getActualTableStructure(context), ConstraintsDescription{}, context);
|
||||||
|
storage->startup();
|
||||||
|
return storage;
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerTableFunctionExecutable(TableFunctionFactory & factory)
|
void registerTableFunctionExecutable(TableFunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction<TableFunctionExecutable>();
|
factory.registerFunction<TableFunctionExecutable>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,28 +1,36 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
#include <DataStreams/IBlockStream_fwd.h>
|
||||||
|
#include <TableFunctions/ITableFunction.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
|
|
||||||
/* executable(path, format, structure, input_query) - creates a temporary storage from executable file
|
/* executable(path, format, structure, input_query) - creates a temporary storage from executable file
|
||||||
*
|
*
|
||||||
*
|
*
|
||||||
* The file must be in the clickhouse data directory.
|
* The file must be in the clickhouse data directory.
|
||||||
* The relative path begins with the clickhouse data directory.
|
* The relative path begins with the clickhouse data directory.
|
||||||
*/
|
*/
|
||||||
class TableFunctionExecutable : public ITableFunctionFileLike
|
class TableFunctionExecutable : public ITableFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = "executable";
|
static constexpr auto name = "executable";
|
||||||
std::string getName() const override
|
std::string getName() const override { return name; }
|
||||||
{
|
bool hasStaticStructure() const override { return true; }
|
||||||
return name;
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
StoragePtr getStorage(
|
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override;
|
||||||
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const override;
|
|
||||||
const char * getStorageTypeName() const override { return "Executable"; }
|
const char * getStorageTypeName() const override { return "Executable"; }
|
||||||
};}
|
|
||||||
|
ColumnsDescription getActualTableStructure(const Context & context) const override;
|
||||||
|
void parseArguments(const ASTPtr & ast_function, const Context & context) override;
|
||||||
|
|
||||||
|
String file_path;
|
||||||
|
String format;
|
||||||
|
String structure;
|
||||||
|
BlockInputStreamPtr input;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
@ -11,6 +11,7 @@ SRCS(
|
|||||||
ITableFunctionFileLike.cpp
|
ITableFunctionFileLike.cpp
|
||||||
ITableFunctionXDBC.cpp
|
ITableFunctionXDBC.cpp
|
||||||
TableFunctionFactory.cpp
|
TableFunctionFactory.cpp
|
||||||
|
TableFunctionExecutable.cpp
|
||||||
TableFunctionFile.cpp
|
TableFunctionFile.cpp
|
||||||
TableFunctionGenerateRandom.cpp
|
TableFunctionGenerateRandom.cpp
|
||||||
TableFunctionInput.cpp
|
TableFunctionInput.cpp
|
||||||
|
Loading…
Reference in New Issue
Block a user