2020-10-14 12:19:29 +00:00
|
|
|
#pragma once
|
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <TableFunctions/ITableFunction.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2020-10-14 12:19:29 +00:00
|
|
|
#include <Storages/StorageProxy.h>
|
|
|
|
#include <Common/CurrentThread.h>
|
2020-11-17 17:16:55 +00:00
|
|
|
#include <Processors/Transforms/ExpressionTransform.h>
|
2020-10-14 12:19:29 +00:00
|
|
|
#include <Interpreters/getHeaderForProcessingStage.h>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int INCOMPATIBLE_COLUMNS;
|
|
|
|
}
|
|
|
|
|
|
|
|
using GetNestedStorageFunc = std::function<StoragePtr()>;
|
|
|
|
|
|
|
|
/// Lazily creates underlying storage.
|
|
|
|
/// Adds ConversionTransform in case of structure mismatch.
|
|
|
|
class StorageTableFunctionProxy final : public StorageProxy
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
StorageTableFunctionProxy(const StorageID & table_id_, GetNestedStorageFunc get_nested_,
|
|
|
|
ColumnsDescription cached_columns, bool add_conversion_ = true)
|
|
|
|
: StorageProxy(table_id_), get_nested(std::move(get_nested_)), add_conversion(add_conversion_)
|
|
|
|
{
|
|
|
|
StorageInMemoryMetadata cached_metadata;
|
|
|
|
cached_metadata.setColumns(std::move(cached_columns));
|
|
|
|
setInMemoryMetadata(cached_metadata);
|
|
|
|
}
|
|
|
|
|
2021-09-03 17:04:30 +00:00
|
|
|
StoragePtr getNestedImpl() const
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock{nested_mutex};
|
|
|
|
if (nested)
|
|
|
|
return nested;
|
|
|
|
|
|
|
|
auto nested_storage = get_nested();
|
|
|
|
nested_storage->startup();
|
|
|
|
nested_storage->renameInMemory(getStorageID());
|
|
|
|
nested = nested_storage;
|
|
|
|
get_nested = {};
|
|
|
|
return nested;
|
|
|
|
}
|
|
|
|
|
2021-09-03 17:04:30 +00:00
|
|
|
StoragePtr getNested() const override
|
|
|
|
{
|
|
|
|
StoragePtr nested_storage = getNestedImpl();
|
|
|
|
assert(!nested_storage->getStoragePolicy());
|
|
|
|
assert(!nested_storage->storesDataOnDisk());
|
|
|
|
return nested_storage;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Table functions cannot have storage policy and cannot store data on disk.
|
|
|
|
/// We may check if table is readonly or stores data on disk on DROP TABLE.
|
|
|
|
/// Avoid loading nested table by returning nullptr/false for all table functions.
|
|
|
|
StoragePolicyPtr getStoragePolicy() const override { return nullptr; }
|
|
|
|
bool storesDataOnDisk() const override { return false; }
|
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
std::lock_guard lock{nested_mutex};
|
|
|
|
if (nested)
|
|
|
|
return nested->getName();
|
|
|
|
return StorageProxy::getName();
|
|
|
|
}
|
|
|
|
|
|
|
|
void startup() override { }
|
|
|
|
void shutdown() override
|
|
|
|
{
|
|
|
|
std::lock_guard lock{nested_mutex};
|
|
|
|
if (nested)
|
|
|
|
nested->shutdown();
|
|
|
|
}
|
|
|
|
|
2021-05-19 17:57:07 +00:00
|
|
|
void flush() override
|
|
|
|
{
|
|
|
|
std::lock_guard lock{nested_mutex};
|
|
|
|
if (nested)
|
|
|
|
nested->flush();
|
|
|
|
}
|
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
void drop() override
|
|
|
|
{
|
|
|
|
std::lock_guard lock{nested_mutex};
|
|
|
|
if (nested)
|
|
|
|
nested->drop();
|
|
|
|
}
|
|
|
|
|
|
|
|
Pipe read(
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-11-07 21:28:39 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-10-14 12:19:29 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
|
|
|
unsigned num_streams) override
|
|
|
|
{
|
|
|
|
String cnames;
|
|
|
|
for (const auto & c : column_names)
|
|
|
|
cnames += c + " ";
|
|
|
|
auto storage = getNested();
|
|
|
|
auto nested_metadata = storage->getInMemoryMetadataPtr();
|
|
|
|
auto pipe = storage->read(column_names, nested_metadata, query_info, context,
|
|
|
|
processed_stage, max_block_size, num_streams);
|
|
|
|
if (!pipe.empty() && add_conversion)
|
|
|
|
{
|
|
|
|
auto to_header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot,
|
|
|
|
query_info, context, processed_stage);
|
2020-11-17 17:16:55 +00:00
|
|
|
|
|
|
|
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
|
|
|
|
pipe.getHeader().getColumnsWithTypeAndName(),
|
|
|
|
to_header.getColumnsWithTypeAndName(),
|
|
|
|
ActionsDAG::MatchColumnsMode::Name);
|
2021-03-04 17:38:12 +00:00
|
|
|
auto convert_actions = std::make_shared<ExpressionActions>(
|
|
|
|
convert_actions_dag,
|
2021-05-19 14:32:07 +00:00
|
|
|
ExpressionActionsSettings::fromSettings(context->getSettingsRef(), CompileExpressions::yes));
|
2020-11-17 17:16:55 +00:00
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
pipe.addSimpleTransform([&](const Block & header)
|
|
|
|
{
|
2020-11-17 17:16:55 +00:00
|
|
|
return std::make_shared<ExpressionTransform>(header, convert_actions);
|
2020-10-14 12:19:29 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
return pipe;
|
|
|
|
}
|
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
SinkToStoragePtr write(
|
2020-10-14 12:19:29 +00:00
|
|
|
const ASTPtr & query,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
|
|
|
auto storage = getNested();
|
|
|
|
auto cached_structure = metadata_snapshot->getSampleBlock();
|
|
|
|
auto actual_structure = storage->getInMemoryMetadataPtr()->getSampleBlock();
|
|
|
|
if (!blocksHaveEqualStructure(actual_structure, cached_structure) && add_conversion)
|
|
|
|
{
|
|
|
|
throw Exception("Source storage and table function have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
|
|
|
|
}
|
|
|
|
return storage->write(query, metadata_snapshot, context);
|
|
|
|
}
|
|
|
|
|
|
|
|
void renameInMemory(const StorageID & new_table_id) override
|
|
|
|
{
|
|
|
|
std::lock_guard lock{nested_mutex};
|
|
|
|
if (nested)
|
|
|
|
StorageProxy::renameInMemory(new_table_id);
|
|
|
|
else
|
|
|
|
IStorage::renameInMemory(new_table_id);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool isView() const override { return false; }
|
|
|
|
void checkTableCanBeDropped() const override {}
|
|
|
|
|
|
|
|
private:
|
|
|
|
mutable std::mutex nested_mutex;
|
|
|
|
mutable GetNestedStorageFunc get_nested;
|
|
|
|
mutable StoragePtr nested;
|
|
|
|
const bool add_conversion;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|