2019-05-28 18:27:00 +00:00
|
|
|
#include <Storages/StorageInput.h>
|
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
|
|
|
|
#include <memory>
|
2022-05-20 19:49:31 +00:00
|
|
|
#include <Processors/ISource.h>
|
2023-11-14 12:51:25 +00:00
|
|
|
#include <Processors/Sources/ThrowingExceptionSource.h>
|
2023-11-13 19:01:36 +00:00
|
|
|
#include <Processors/QueryPlan/ISourceStep.h>
|
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2023-11-13 19:01:36 +00:00
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
2019-05-28 18:27:00 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2019-09-05 13:17:01 +00:00
|
|
|
extern const int INVALID_USAGE_OF_INPUT;
|
2023-11-14 14:55:21 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2019-05-28 18:27:00 +00:00
|
|
|
}
|
|
|
|
|
2020-03-10 19:36:17 +00:00
|
|
|
StorageInput::StorageInput(const StorageID & table_id, const ColumnsDescription & columns_)
|
|
|
|
: IStorage(table_id)
|
2019-05-28 18:27:00 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
2019-05-28 18:27:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-05-20 19:49:31 +00:00
|
|
|
class StorageInputSource : public ISource, WithContext
|
2019-05-28 18:27:00 +00:00
|
|
|
{
|
|
|
|
public:
|
2022-05-20 19:49:31 +00:00
|
|
|
StorageInputSource(ContextPtr context_, Block sample_block) : ISource(std::move(sample_block)), WithContext(context_) {}
|
2019-05-28 18:27:00 +00:00
|
|
|
|
2020-01-31 14:26:51 +00:00
|
|
|
Chunk generate() override
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
auto block = getContext()->getInputBlocksReaderCallback()(getContext());
|
2020-01-31 14:26:51 +00:00
|
|
|
if (!block)
|
|
|
|
return {};
|
2019-05-28 18:27:00 +00:00
|
|
|
|
2020-01-31 14:26:51 +00:00
|
|
|
UInt64 num_rows = block.rows();
|
|
|
|
return Chunk(block.getColumns(), num_rows);
|
|
|
|
}
|
2019-05-28 18:27:00 +00:00
|
|
|
|
2020-01-31 14:26:51 +00:00
|
|
|
String getName() const override { return "Input"; }
|
2019-05-28 18:27:00 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2021-07-20 18:18:43 +00:00
|
|
|
void StorageInput::setPipe(Pipe pipe_)
|
2019-05-28 18:27:00 +00:00
|
|
|
{
|
2021-07-20 18:18:43 +00:00
|
|
|
pipe = std::move(pipe_);
|
2023-11-13 19:01:36 +00:00
|
|
|
was_pipe_initialized = true;
|
2019-05-28 18:27:00 +00:00
|
|
|
}
|
|
|
|
|
2023-11-13 19:01:36 +00:00
|
|
|
class ReadFromInput : public ISourceStep
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
std::string getName() const override { return "ReadFromInput"; }
|
|
|
|
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
|
|
|
|
|
|
|
ReadFromInput(
|
|
|
|
Block sample_block,
|
2023-11-14 12:51:25 +00:00
|
|
|
Pipe pipe_,
|
2023-11-13 19:01:36 +00:00
|
|
|
StorageInput & storage_)
|
|
|
|
: ISourceStep(DataStream{.header = std::move(sample_block)})
|
2023-11-14 12:51:25 +00:00
|
|
|
, pipe(std::move(pipe_))
|
2023-11-13 19:01:36 +00:00
|
|
|
, storage(storage_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
2023-11-14 12:51:25 +00:00
|
|
|
Pipe pipe;
|
2023-11-13 19:01:36 +00:00
|
|
|
StorageInput & storage;
|
|
|
|
};
|
2019-05-28 18:27:00 +00:00
|
|
|
|
2023-11-13 19:01:36 +00:00
|
|
|
void StorageInput::read(
|
|
|
|
QueryPlan & query_plan,
|
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2019-05-28 18:27:00 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t /*max_block_size*/,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t /*num_streams*/)
|
2019-05-28 18:27:00 +00:00
|
|
|
{
|
2023-11-13 19:01:36 +00:00
|
|
|
storage_snapshot->check(column_names);
|
|
|
|
Block sample_block = storage_snapshot->metadata->getSampleBlock();
|
2023-11-14 12:51:25 +00:00
|
|
|
Pipe input_source_pipe;
|
2023-11-13 19:01:36 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context = context->getQueryContext();
|
2019-05-28 18:27:00 +00:00
|
|
|
/// It is TCP request if we have callbacks for input().
|
2023-11-14 12:51:25 +00:00
|
|
|
if (query_context->getInputBlocksReaderCallback())
|
2019-05-28 18:27:00 +00:00
|
|
|
{
|
|
|
|
/// Send structure to the client.
|
2021-04-10 23:33:54 +00:00
|
|
|
query_context->initializeInput(shared_from_this());
|
2023-11-14 12:51:25 +00:00
|
|
|
input_source_pipe = Pipe(std::make_shared<StorageInputSource>(query_context, sample_block));
|
2019-05-28 18:27:00 +00:00
|
|
|
}
|
|
|
|
|
2023-11-13 19:01:36 +00:00
|
|
|
auto reading = std::make_unique<ReadFromInput>(
|
|
|
|
std::move(sample_block),
|
2023-11-14 12:51:25 +00:00
|
|
|
std::move(input_source_pipe),
|
2023-11-13 19:01:36 +00:00
|
|
|
*this);
|
|
|
|
|
|
|
|
query_plan.addStep(std::move(reading));
|
|
|
|
}
|
|
|
|
|
|
|
|
void ReadFromInput::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
|
|
|
{
|
2023-11-14 12:51:25 +00:00
|
|
|
if (!pipe.empty())
|
|
|
|
{
|
|
|
|
pipeline.init(std::move(pipe));
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!storage.was_pipe_initialized)
|
|
|
|
throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "Input stream is not initialized, input() must be used only in INSERT SELECT query");
|
|
|
|
|
2023-11-13 19:01:36 +00:00
|
|
|
if (storage.was_pipe_used)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to read from input() twice.");
|
|
|
|
|
|
|
|
pipeline.init(std::move(storage.pipe));
|
|
|
|
storage.was_pipe_used = true;
|
2019-05-28 18:27:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|