ClickHouse/src/Storages/StorageInput.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

79 lines
2.0 KiB
C++
Raw Normal View History

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>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/Pipe.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;
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
Chunk generate() override
{
auto block = getContext()->getInputBlocksReaderCallback()(getContext());
if (!block)
return {};
2019-05-28 18:27:00 +00:00
UInt64 num_rows = block.rows();
return Chunk(block.getColumns(), num_rows);
}
2019-05-28 18:27:00 +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_);
2019-05-28 18:27:00 +00:00
}
2020-08-03 13:54:14 +00:00
Pipe StorageInput::read(
const Names & /*column_names*/,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & /*query_info*/,
ContextPtr context,
2019-05-28 18:27:00 +00:00
QueryProcessingStage::Enum /*processed_stage*/,
size_t /*max_block_size*/,
size_t /*num_streams*/)
2019-05-28 18:27:00 +00:00
{
Pipes pipes;
auto query_context = context->getQueryContext();
2019-05-28 18:27:00 +00:00
/// It is TCP request if we have callbacks for input().
if (query_context->getInputBlocksReaderCallback())
2019-05-28 18:27:00 +00:00
{
/// Send structure to the client.
query_context->initializeInput(shared_from_this());
return Pipe(std::make_shared<StorageInputSource>(query_context, storage_snapshot->metadata->getSampleBlock()));
2019-05-28 18:27:00 +00:00
}
2021-07-20 18:18:43 +00:00
if (pipe.empty())
throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "Input stream is not initialized, input() must be used only in INSERT SELECT query");
2019-05-28 18:27:00 +00:00
2021-07-20 18:18:43 +00:00
return std::move(pipe);
2019-05-28 18:27:00 +00:00
}
}