2021-04-14 17:51:55 +00:00
|
|
|
#include <Storages/StorageExecutable.h>
|
|
|
|
#include <Processors/Pipe.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Storages/StorageFile.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <IO/CompressionMethod.h>
|
2021-04-15 09:40:41 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
2021-04-14 17:51:55 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Processors/Sources/SourceWithProgress.h>
|
2021-04-15 21:15:54 +00:00
|
|
|
#include <DataStreams/ShellCommandBlockInputStream.h>
|
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2021-04-14 17:51:55 +00:00
|
|
|
#include <Dictionaries/ExecutableDictionarySource.h>
|
2021-04-15 09:40:41 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
2021-04-14 17:51:55 +00:00
|
|
|
#include <Core/Block.h>
|
2021-04-15 09:40:41 +00:00
|
|
|
#include <Common/ShellCommand.h>
|
2021-04-14 17:51:55 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int UNKNOWN_IDENTIFIER;
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
class StorageExecutableSource : public SourceWithProgress
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
|
|
|
String getName() const override { return "Executable"; }
|
|
|
|
|
|
|
|
StorageExecutableSource(
|
2021-04-15 21:15:54 +00:00
|
|
|
const String & file_path_,
|
2021-04-15 09:40:41 +00:00
|
|
|
const String & format_,
|
2021-04-15 21:15:54 +00:00
|
|
|
BlockInputStreamPtr input_,
|
2021-04-15 09:40:41 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
|
|
|
const Context & context_,
|
2021-04-15 21:15:54 +00:00
|
|
|
UInt64 max_block_size_)
|
2021-04-15 09:40:41 +00:00
|
|
|
: SourceWithProgress(metadata_snapshot_->getSampleBlock())
|
2021-04-14 17:51:55 +00:00
|
|
|
, file_path(std::move(file_path_))
|
2021-04-15 21:15:54 +00:00
|
|
|
, format(format_)
|
|
|
|
, input(input_)
|
2021-04-15 09:40:41 +00:00
|
|
|
, metadata_snapshot(metadata_snapshot_)
|
|
|
|
, context(context_)
|
|
|
|
, max_block_size(max_block_size_)
|
2021-04-14 17:51:55 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
Chunk generate() override {
|
2021-04-15 09:40:41 +00:00
|
|
|
LOG_TRACE(log, "generating {}", toString(file_path));
|
|
|
|
|
|
|
|
if (!reader)
|
|
|
|
{
|
2021-04-15 21:15:54 +00:00
|
|
|
auto sample_block = metadata_snapshot->getSampleBlock();
|
|
|
|
reader = std::make_shared<BlockInputStreamWithBackgroundThread>(context, format, sample_block, file_path, log, max_block_size, [this](WriteBufferFromFile & out)
|
|
|
|
{
|
|
|
|
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();
|
|
|
|
});
|
2021-04-15 09:40:41 +00:00
|
|
|
|
|
|
|
reader->readPrefix();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (auto res = reader->read())
|
|
|
|
{
|
|
|
|
Columns columns = res.getColumns();
|
|
|
|
UInt64 num_rows = res.rows();
|
|
|
|
return Chunk(std::move(columns), num_rows);
|
|
|
|
}
|
|
|
|
|
|
|
|
reader->readSuffix();
|
|
|
|
reader.reset();
|
|
|
|
return {};
|
2021-04-14 17:51:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
String file_path;
|
2021-04-15 21:15:54 +00:00
|
|
|
const String & format;
|
|
|
|
BlockInputStreamPtr input;
|
2021-04-15 09:40:41 +00:00
|
|
|
BlockInputStreamPtr reader;
|
|
|
|
const StorageMetadataPtr & metadata_snapshot;
|
|
|
|
const Context & context;
|
|
|
|
UInt64 max_block_size;
|
|
|
|
Poco::Logger * log = &Poco::Logger::get("StorageExecutableSource");
|
2021-04-14 17:51:55 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
StorageExecutable::StorageExecutable(
|
2021-04-15 09:40:41 +00:00
|
|
|
const StorageID & table_id_,
|
2021-04-14 17:51:55 +00:00
|
|
|
const String & file_path_,
|
2021-04-15 21:15:54 +00:00
|
|
|
const String & format_,
|
|
|
|
BlockInputStreamPtr input_,
|
2021-04-14 17:51:55 +00:00
|
|
|
const ColumnsDescription & columns,
|
|
|
|
const ConstraintsDescription & constraints,
|
2021-04-15 21:15:54 +00:00
|
|
|
const Context & context_)
|
2021-04-15 09:40:41 +00:00
|
|
|
: IStorage(table_id_)
|
2021-04-14 17:51:55 +00:00
|
|
|
, file_path(file_path_)
|
2021-04-15 21:15:54 +00:00
|
|
|
, format(format_)
|
|
|
|
, input(input_)
|
2021-04-14 17:51:55 +00:00
|
|
|
, context(context_)
|
|
|
|
{
|
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns);
|
|
|
|
storage_metadata.setConstraints(constraints);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
|
|
|
}
|
|
|
|
|
|
|
|
Pipe StorageExecutable::read(
|
|
|
|
const Names & /*column_names*/,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
SelectQueryInfo & /*query_info*/,
|
|
|
|
const Context & context_,
|
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t max_block_size,
|
|
|
|
unsigned /*num_streams*/)
|
|
|
|
{
|
|
|
|
return Pipe(std::make_shared<StorageExecutableSource>(
|
2021-04-15 21:15:54 +00:00
|
|
|
file_path,
|
|
|
|
format,
|
|
|
|
input,
|
2021-04-15 09:40:41 +00:00
|
|
|
metadata_snapshot,
|
2021-04-14 17:51:55 +00:00
|
|
|
context_,
|
2021-04-15 21:15:54 +00:00
|
|
|
max_block_size));
|
2021-04-14 17:51:55 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|