ClickHouse/src/TableFunctions/TableFunctionLoop.cpp

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

145 lines
4.7 KiB
C++
Raw Normal View History

2024-04-23 12:54:23 +00:00
#include "config.h"
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
2024-04-24 08:11:14 +00:00
#include <Databases/IDatabase.h>
2024-04-23 12:54:23 +00:00
#include <Parsers/ASTFunction.h>
2024-04-24 08:11:14 +00:00
#include <Parsers/ASTIdentifier.h>
2024-04-23 12:54:23 +00:00
#include <Common/Exception.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Storages/checkAndGetLiteralArgument.h>
2024-04-24 08:11:14 +00:00
#include <Storages/StorageLoop.h>
2024-04-23 12:54:23 +00:00
#include "registerTableFunctions.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int UNKNOWN_TABLE;
}
namespace
{
class TableFunctionLoop : public ITableFunction{
public:
static constexpr auto name = "loop";
std::string getName() const override { return name; }
private:
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
const char * getStorageTypeName() const override { return "Loop"; }
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
// save the inner table function AST
ASTPtr inner_table_function_ast;
// save database and table
std::string database_name_;
std::string table_name_;
};
}
void TableFunctionLoop::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
const auto & args_func = ast_function->as<ASTFunction &>();
if (!args_func.arguments)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function 'loop' must have arguments.");
auto & args = args_func.arguments->children;
if (args.empty())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "No arguments provided for table function 'loop'");
2024-04-24 08:11:14 +00:00
if (args.size() == 1)
{
if (const auto * id = args[0]->as<ASTIdentifier>())
{
String id_name = id->name();
size_t dot_pos = id_name.find('.');
if (dot_pos != String::npos)
{
database_name_ = id_name.substr(0, dot_pos);
table_name_ = id_name.substr(dot_pos + 1);
}
else
{
table_name_ = id_name;
}
}
else if (const auto * func = args[0]->as<ASTFunction>())
{
inner_table_function_ast = args[0];
}
else
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Expected identifier or function for argument 1 of function 'loop', got {}", args[0]->getID());
}
}
2024-04-23 12:54:23 +00:00
// loop(database, table)
2024-04-24 08:11:14 +00:00
else if (args.size() == 2)
2024-04-23 12:54:23 +00:00
{
args[0] = evaluateConstantExpressionForDatabaseName(args[0], context);
args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], context);
database_name_ = checkAndGetLiteralArgument<String>(args[0], "database");
table_name_ = checkAndGetLiteralArgument<String>(args[1], "table");
}
else
2024-04-24 08:11:14 +00:00
{
2024-04-23 12:54:23 +00:00
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function 'loop' must have 1 or 2 arguments.");
2024-04-24 08:11:14 +00:00
}
2024-04-23 12:54:23 +00:00
}
2024-04-29 11:44:44 +00:00
ColumnsDescription TableFunctionLoop::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const
2024-04-23 12:54:23 +00:00
{
2024-04-29 11:44:44 +00:00
return ColumnsDescription();
2024-04-23 12:54:23 +00:00
}
StoragePtr TableFunctionLoop::executeImpl(
const ASTPtr & /*ast_function*/,
ContextPtr context,
const std::string & table_name,
ColumnsDescription cached_columns,
bool is_insert_query) const
{
StoragePtr storage;
2024-04-24 08:11:14 +00:00
if (!table_name_.empty())
2024-04-23 12:54:23 +00:00
{
2024-04-24 08:11:14 +00:00
String database_name = database_name_;
if (database_name.empty())
database_name = context->getCurrentDatabase();
auto database = DatabaseCatalog::instance().getDatabase(database_name);
2024-04-23 12:54:23 +00:00
storage = database->tryGetTable(table_name_ ,context);
if (!storage)
2024-04-24 08:11:14 +00:00
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table '{}' not found in database '{}'", table_name_, database_name);
2024-04-23 12:54:23 +00:00
}
2024-04-24 08:11:14 +00:00
2024-04-23 12:54:23 +00:00
else
{
auto inner_table_function = TableFunctionFactory::instance().get(inner_table_function_ast, context);
storage = inner_table_function->execute(
inner_table_function_ast,
context,
table_name,
std::move(cached_columns),
is_insert_query);
}
2024-04-24 08:11:14 +00:00
auto res = std::make_shared<StorageLoop>(
StorageID(getDatabaseName(), table_name),
storage
);
res->startup();
return res;
2024-04-23 12:54:23 +00:00
}
void registerTableFunctionLoop(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionLoop>();
}
}