ClickHouse/src/TableFunctions/TableFunctionDictionary.cpp
Robert Schulze 330212e0f4
Remove inherited create() method + disallow copying
The original motivation for this commit was that shared_ptr_helper used
std::shared_ptr<>() which does two heap allocations instead of
make_shared<>() which does a single allocation. Turned out that
1. the affected code (--> Storages/) is not on a hot path (rendering the
performance argument moot ...)
2. yet copying Storage objects is potentially dangerous and was
   previously allowed.

Hence, this change

- removes shared_ptr_helper and as a result all inherited create() methods,

- instead, Storage objects are now created using make_shared<>() by the
  caller (for that to work, many constructors had to be made public), and

- all Storage classes were marked as noncopyable using boost::noncopyable.

In sum, we are (likely) not making things faster but the code becomes
cleaner and harder to misuse.
2022-05-02 08:46:52 +02:00

68 lines
2.1 KiB
C++

#include <TableFunctions/TableFunctionDictionary.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Storages/StorageDictionary.h>
#include <TableFunctions/TableFunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
// Parse args
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function ({}) must have arguments.", quoteString(getName()));
ASTs & args = args_func.at(0)->children;
if (args.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function ({}) requires 1 arguments", quoteString(getName()));
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
dictionary_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
}
ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const
{
const ExternalDictionariesLoader & external_loader = context->getExternalDictionariesLoader();
auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context);
auto result = ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure));
return result;
}
StoragePtr TableFunctionDictionary::executeImpl(
const ASTPtr &, ContextPtr context, const std::string & table_name, ColumnsDescription) const
{
StorageID dict_id(getDatabaseName(), table_name);
auto dictionary_table_structure = getActualTableStructure(context);
auto result = std::make_shared<StorageDictionary>(
dict_id, dictionary_name, std::move(dictionary_table_structure), String{}, StorageDictionary::Location::Custom, context);
return result;
}
void registerTableFunctionDictionary(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionDictionary>();
}
}