mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Remove useless header files
This commit is contained in:
parent
0f9e7594e9
commit
1c318891b8
@ -1,12 +1,13 @@
|
||||
#include <TableFunctions/Hive/TableFunctionHive.h>
|
||||
#include "config.h"
|
||||
|
||||
#if USE_HIVE
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <memory>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -15,75 +16,111 @@
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
void TableFunctionHive::parseArguments(const ASTPtr & ast_function_, ContextPtr context_)
|
||||
{
|
||||
ASTs & args_func = ast_function_->children;
|
||||
if (args_func.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName());
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
if (args.size() != 5)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"The signature of function {} is:\n - hive_url, hive_database, hive_table, structure, partition_by_keys",
|
||||
getName());
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_);
|
||||
|
||||
hive_metastore_url = checkAndGetLiteralArgument<String>(args[0], "hive_url");
|
||||
hive_database = checkAndGetLiteralArgument<String>(args[1], "hive_database");
|
||||
hive_table = checkAndGetLiteralArgument<String>(args[2], "hive_table");
|
||||
table_structure = checkAndGetLiteralArgument<String>(args[3], "structure");
|
||||
partition_by_def = checkAndGetLiteralArgument<String>(args[4], "partition_by_keys");
|
||||
|
||||
actual_columns = parseColumnsListFromString(table_structure, context_);
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionHive::getActualTableStructure(ContextPtr /*context_*/, bool /*is_insert_query*/) const { return actual_columns; }
|
||||
|
||||
StoragePtr TableFunctionHive::executeImpl(
|
||||
const ASTPtr & /*ast_function_*/,
|
||||
ContextPtr context_,
|
||||
const std::string & table_name_,
|
||||
ColumnsDescription /*cached_columns_*/,
|
||||
bool /*is_insert_query*/) const
|
||||
{
|
||||
const Settings & settings = context_->getSettings();
|
||||
ParserExpression partition_by_parser;
|
||||
ASTPtr partition_by_ast = parseQuery(
|
||||
partition_by_parser,
|
||||
"(" + partition_by_def + ")",
|
||||
"partition by declaration list",
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth);
|
||||
StoragePtr storage;
|
||||
storage = std::make_shared<StorageHive>(
|
||||
hive_metastore_url,
|
||||
hive_database,
|
||||
hive_table,
|
||||
StorageID(getDatabaseName(), table_name_),
|
||||
actual_columns,
|
||||
ConstraintsDescription{},
|
||||
"",
|
||||
partition_by_ast,
|
||||
std::make_unique<HiveSettings>(),
|
||||
context_);
|
||||
|
||||
return storage;
|
||||
}
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionHive(TableFunctionFactory & factory_) { factory_.registerFunction<TableFunctionHive>(); }
|
||||
namespace
|
||||
{
|
||||
|
||||
class TableFunctionHive : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hive";
|
||||
static constexpr auto storage_type_name = "Hive";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return storage_type_name; }
|
||||
ColumnsDescription getActualTableStructure(ContextPtr, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function_, ContextPtr context_) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * logger = &Poco::Logger::get("TableFunctionHive");
|
||||
|
||||
String cluster_name;
|
||||
String hive_metastore_url;
|
||||
String hive_database;
|
||||
String hive_table;
|
||||
String table_structure;
|
||||
String partition_by_def;
|
||||
|
||||
ColumnsDescription actual_columns;
|
||||
};
|
||||
|
||||
void TableFunctionHive::parseArguments(const ASTPtr & ast_function_, ContextPtr context_)
|
||||
{
|
||||
ASTs & args_func = ast_function_->children;
|
||||
if (args_func.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName());
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
if (args.size() != 5)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"The signature of function {} is:\n - hive_url, hive_database, hive_table, structure, partition_by_keys",
|
||||
getName());
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_);
|
||||
|
||||
hive_metastore_url = checkAndGetLiteralArgument<String>(args[0], "hive_url");
|
||||
hive_database = checkAndGetLiteralArgument<String>(args[1], "hive_database");
|
||||
hive_table = checkAndGetLiteralArgument<String>(args[2], "hive_table");
|
||||
table_structure = checkAndGetLiteralArgument<String>(args[3], "structure");
|
||||
partition_by_def = checkAndGetLiteralArgument<String>(args[4], "partition_by_keys");
|
||||
|
||||
actual_columns = parseColumnsListFromString(table_structure, context_);
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionHive::getActualTableStructure(ContextPtr /*context_*/, bool /*is_insert_query*/) const { return actual_columns; }
|
||||
|
||||
StoragePtr TableFunctionHive::executeImpl(
|
||||
const ASTPtr & /*ast_function_*/,
|
||||
ContextPtr context_,
|
||||
const std::string & table_name_,
|
||||
ColumnsDescription /*cached_columns_*/,
|
||||
bool /*is_insert_query*/) const
|
||||
{
|
||||
const Settings & settings = context_->getSettings();
|
||||
ParserExpression partition_by_parser;
|
||||
ASTPtr partition_by_ast = parseQuery(
|
||||
partition_by_parser,
|
||||
"(" + partition_by_def + ")",
|
||||
"partition by declaration list",
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth);
|
||||
StoragePtr storage;
|
||||
storage = std::make_shared<StorageHive>(
|
||||
hive_metastore_url,
|
||||
hive_database,
|
||||
hive_table,
|
||||
StorageID(getDatabaseName(), table_name_),
|
||||
actual_columns,
|
||||
ConstraintsDescription{},
|
||||
"",
|
||||
partition_by_ast,
|
||||
std::make_unique<HiveSettings>(),
|
||||
context_);
|
||||
|
||||
return storage;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionHive(TableFunctionFactory & factory_) { factory_.registerFunction<TableFunctionHive>(); }
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,39 +0,0 @@
|
||||
#pragma once
|
||||
#include "config.h"
|
||||
|
||||
#if USE_HIVE
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Poco/Logger.h>
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
class TableFunctionHive : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hive";
|
||||
static constexpr auto storage_type_name = "Hive";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return storage_type_name; }
|
||||
ColumnsDescription getActualTableStructure(ContextPtr, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function_, ContextPtr context_) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * logger = &Poco::Logger::get("TableFunctionHive");
|
||||
|
||||
String cluster_name;
|
||||
String hive_metastore_url;
|
||||
String hive_database;
|
||||
String hive_table;
|
||||
String table_structure;
|
||||
String partition_by_def;
|
||||
|
||||
ColumnsDescription actual_columns;
|
||||
};
|
||||
}
|
||||
#endif
|
@ -9,12 +9,16 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/StorageXDBC.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/ITableFunctionXDBC.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Common/Exception.h>
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <BridgeHelper/XDBCBridgeHelper.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,6 +28,79 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/**
|
||||
* Base class for table functions, that works over external bridge
|
||||
* Xdbc (Xdbc connect string, table) - creates a temporary StorageXDBC.
|
||||
*/
|
||||
class ITableFunctionXDBC : public ITableFunction
|
||||
{
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
/* A factory method to create bridge helper, that will assist in remote interaction */
|
||||
virtual BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_) const = 0;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
void startBridgeIfNot(ContextPtr context) const;
|
||||
|
||||
String connection_string;
|
||||
String schema_name;
|
||||
String remote_table_name;
|
||||
mutable BridgeHelperPtr helper;
|
||||
};
|
||||
|
||||
class TableFunctionJDBC : public ITableFunctionXDBC
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "jdbc";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
private:
|
||||
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_) const override
|
||||
{
|
||||
return std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(context, http_timeout_, connection_string_, use_connection_pooling_);
|
||||
}
|
||||
|
||||
const char * getStorageTypeName() const override { return "JDBC"; }
|
||||
};
|
||||
|
||||
class TableFunctionODBC : public ITableFunctionXDBC
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "odbc";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
private:
|
||||
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_) const override
|
||||
{
|
||||
return std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(context, http_timeout_, connection_string_, use_connection_pooling_);
|
||||
}
|
||||
|
||||
const char * getStorageTypeName() const override { return "ODBC"; }
|
||||
};
|
||||
|
||||
|
||||
void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
const auto & args_func = ast_function->as<ASTFunction &>();
|
||||
@ -102,6 +179,8 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, Cont
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionJDBC(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionJDBC>();
|
||||
|
@ -1,80 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/StorageXDBC.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <BridgeHelper/XDBCBridgeHelper.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/**
|
||||
* Base class for table functions, that works over external bridge
|
||||
* Xdbc (Xdbc connect string, table) - creates a temporary StorageXDBC.
|
||||
*/
|
||||
class ITableFunctionXDBC : public ITableFunction
|
||||
{
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
/* A factory method to create bridge helper, that will assist in remote interaction */
|
||||
virtual BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_) const = 0;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
void startBridgeIfNot(ContextPtr context) const;
|
||||
|
||||
String connection_string;
|
||||
String schema_name;
|
||||
String remote_table_name;
|
||||
mutable BridgeHelperPtr helper;
|
||||
};
|
||||
|
||||
class TableFunctionJDBC : public ITableFunctionXDBC
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "jdbc";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
private:
|
||||
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_) const override
|
||||
{
|
||||
return std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(context, http_timeout_, connection_string_, use_connection_pooling_);
|
||||
}
|
||||
|
||||
const char * getStorageTypeName() const override { return "JDBC"; }
|
||||
};
|
||||
|
||||
class TableFunctionODBC : public ITableFunctionXDBC
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "odbc";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
private:
|
||||
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||
Poco::Timespan http_timeout_,
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_) const override
|
||||
{
|
||||
return std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(context, http_timeout_, connection_string_, use_connection_pooling_);
|
||||
}
|
||||
|
||||
const char * getStorageTypeName() const override { return "ODBC"; }
|
||||
};
|
||||
}
|
@ -6,7 +6,6 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
@ -15,6 +14,7 @@
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -87,6 +87,7 @@ StoragePtr TableFunctionDictionary::executeImpl(
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionDictionary(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionDictionary>();
|
||||
|
@ -27,4 +27,6 @@ public:
|
||||
private:
|
||||
String dictionary_name;
|
||||
ColumnsDescription dictionary_columns;
|
||||
};}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <TableFunctions/TableFunctionExecutable.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
@ -12,9 +11,7 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Storages/StorageExecutable.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
@ -30,6 +27,44 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* executable(script_name_optional_arguments, format, structure, input_query) - creates a temporary storage from executable file
|
||||
*
|
||||
*
|
||||
* The file must be in the clickhouse data directory.
|
||||
* The relative path begins with the clickhouse data directory.
|
||||
*/
|
||||
class TableFunctionExecutable : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "executable";
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "Executable"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String script_name;
|
||||
std::vector<String> arguments;
|
||||
String format;
|
||||
String structure;
|
||||
std::vector<ASTPtr> input_queries;
|
||||
ASTPtr settings_query = nullptr;
|
||||
};
|
||||
|
||||
|
||||
std::vector<size_t> TableFunctionExecutable::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const
|
||||
{
|
||||
const auto & table_function_node = query_node_table_function->as<TableFunctionNode &>();
|
||||
@ -140,6 +175,8 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
return storage;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionExecutable(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionExecutable>();
|
||||
|
@ -1,44 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class ASTSetQuery;
|
||||
|
||||
/* executable(script_name_optional_arguments, format, structure, input_query) - creates a temporary storage from executable file
|
||||
*
|
||||
*
|
||||
* The file must be in the clickhouse data directory.
|
||||
* The relative path begins with the clickhouse data directory.
|
||||
*/
|
||||
class TableFunctionExecutable : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "executable";
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "Executable"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String script_name;
|
||||
std::vector<String> arguments;
|
||||
String format;
|
||||
String structure;
|
||||
std::vector<ASTPtr> input_queries;
|
||||
ASTPtr settings_query = nullptr;
|
||||
};
|
||||
}
|
@ -1,27 +1,54 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionExplain.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Interpreters/InterpreterSetQuery.h>
|
||||
#include <Interpreters/InterpreterExplainQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class TableFunctionExplain : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "viewExplain";
|
||||
|
||||
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 "Explain"; }
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
InterpreterExplainQuery getInterpreter(ContextPtr context) const;
|
||||
|
||||
ASTPtr query = nullptr;
|
||||
};
|
||||
|
||||
std::vector<size_t> TableFunctionExplain::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr /*context*/) const
|
||||
{
|
||||
const auto & table_function_node = query_node_table_function->as<TableFunctionNode &>();
|
||||
@ -145,6 +172,8 @@ InterpreterExplainQuery TableFunctionExplain::getInterpreter(ContextPtr context)
|
||||
return InterpreterExplainQuery(query, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionExplain(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionExplain>({.documentation = {
|
||||
|
@ -1,35 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Parsers/ASTExplainQuery.h>
|
||||
#include <Interpreters/InterpreterExplainQuery.h>
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableFunctionExplain : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "viewExplain";
|
||||
|
||||
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 "Explain"; }
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
InterpreterExplainQuery getInterpreter(ContextPtr context) const;
|
||||
|
||||
ASTPtr query = nullptr;
|
||||
};
|
||||
|
||||
}
|
@ -1,5 +1,5 @@
|
||||
#include <TableFunctions/TableFunctionFile.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
|
||||
#include "Parsers/IAST_fwd.h"
|
||||
#include "registerTableFunctions.h"
|
||||
@ -12,6 +12,7 @@
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -20,6 +21,42 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* file(path, format[, structure, compression]) - creates a temporary storage from file
|
||||
*
|
||||
* The file must be in the clickhouse data directory.
|
||||
* The relative path begins with the clickhouse data directory.
|
||||
*/
|
||||
class TableFunctionFile : public ITableFunctionFileLike
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "file";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
std::unordered_set<String> getVirtualsToCheckBeforeUsingStructureHint() const override
|
||||
{
|
||||
return {"_path", "_file"};
|
||||
}
|
||||
|
||||
protected:
|
||||
int fd = -1;
|
||||
void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) override;
|
||||
String getFormatFromFirstArgument() override;
|
||||
|
||||
private:
|
||||
StoragePtr getStorage(
|
||||
const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context,
|
||||
const std::string & table_name, const std::string & compression_method_) const override;
|
||||
const char * getStorageTypeName() const override { return "File"; }
|
||||
};
|
||||
|
||||
void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr & context)
|
||||
{
|
||||
if (context->getApplicationType() != Context::ApplicationType::LOCAL)
|
||||
@ -111,6 +148,8 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context
|
||||
return parseColumnsListFromString(structure, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionFile(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionFile>();
|
||||
|
@ -1,42 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* file(path, format[, structure, compression]) - creates a temporary storage from file
|
||||
*
|
||||
* The file must be in the clickhouse data directory.
|
||||
* The relative path begins with the clickhouse data directory.
|
||||
*/
|
||||
class TableFunctionFile : public ITableFunctionFileLike
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "file";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
std::unordered_set<String> getVirtualsToCheckBeforeUsingStructureHint() const override
|
||||
{
|
||||
return {"_path", "_file"};
|
||||
}
|
||||
|
||||
protected:
|
||||
int fd = -1;
|
||||
void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) override;
|
||||
String getFormatFromFirstArgument() override;
|
||||
|
||||
private:
|
||||
StoragePtr getStorage(
|
||||
const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context,
|
||||
const std::string & table_name, const std::string & compression_method_) const override;
|
||||
const char * getStorageTypeName() const override { return "File"; }
|
||||
};
|
||||
|
||||
}
|
@ -18,7 +18,6 @@
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFormat.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
@ -31,6 +30,32 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* format(format_name, data) - ...
|
||||
*/
|
||||
class TableFunctionFormat : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "format";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return false; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Values"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
Block parseData(ColumnsDescription columns, ContextPtr context) const;
|
||||
|
||||
String format;
|
||||
String data;
|
||||
String structure = "auto";
|
||||
};
|
||||
|
||||
void TableFunctionFormat::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
ASTs & args_func = ast_function->children;
|
||||
@ -104,7 +129,7 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con
|
||||
return res;
|
||||
}
|
||||
|
||||
static const FunctionDocumentation format_table_function_documentation =
|
||||
const FunctionDocumentation format_table_function_documentation =
|
||||
{
|
||||
.description=R"(
|
||||
Extracts table structure from data and parses it according to specified input format.
|
||||
@ -168,8 +193,12 @@ Result:
|
||||
.categories{"format", "table-functions"}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionFormat(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionFormat>({format_table_function_documentation, false}, TableFunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,34 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/* format(format_name, data) - ...
|
||||
*/
|
||||
class TableFunctionFormat : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "format";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return false; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Values"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
Block parseData(ColumnsDescription columns, ContextPtr context) const;
|
||||
|
||||
String format;
|
||||
String data;
|
||||
String structure = "auto";
|
||||
};
|
||||
|
||||
}
|
@ -8,7 +8,6 @@
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionGenerateRandom.h>
|
||||
#include <Functions/FunctionGenerateRandomStructure.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -28,6 +27,36 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* generateRandom([structure, max_array_length, max_string_length, random_seed])
|
||||
* - creates a temporary storage that generates columns with random data
|
||||
*/
|
||||
class TableFunctionGenerateRandom : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "generateRandom";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return structure != "auto"; }
|
||||
|
||||
bool needStructureHint() const override { return structure == "auto"; }
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "GenerateRandom"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String structure = "auto";
|
||||
UInt64 max_string_length = 10;
|
||||
UInt64 max_array_length = 10;
|
||||
std::optional<UInt64> random_seed;
|
||||
ColumnsDescription structure_hint;
|
||||
};
|
||||
|
||||
void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
ASTs & args_func = ast_function->children;
|
||||
@ -122,11 +151,11 @@ StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_functio
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionGenerate(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionGenerateRandom>({.documentation = {}, .allow_readonly = true});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,36 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* generateRandom([structure, max_array_length, max_string_length, random_seed])
|
||||
* - creates a temporary storage that generates columns with random data
|
||||
*/
|
||||
class TableFunctionGenerateRandom : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "generateRandom";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return structure != "auto"; }
|
||||
|
||||
bool needStructureHint() const override { return structure == "auto"; }
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "GenerateRandom"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String structure = "auto";
|
||||
UInt64 max_string_length = 10;
|
||||
UInt64 max_array_length = 10;
|
||||
std::optional<UInt64> random_seed;
|
||||
ColumnsDescription structure_hint;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -1,4 +1,3 @@
|
||||
#include <TableFunctions/TableFunctionInput.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -21,6 +20,31 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* input(structure) - allows to make INSERT SELECT from incoming stream of data
|
||||
*/
|
||||
class TableFunctionInput : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "input";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
bool needStructureHint() const override { return true; }
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Input"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String structure;
|
||||
ColumnsDescription structure_hint;
|
||||
};
|
||||
|
||||
void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
const auto * function = ast_function->as<ASTFunction>();
|
||||
@ -65,6 +89,8 @@ StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, Cont
|
||||
return storage;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionInput(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionInput>();
|
||||
|
@ -1,33 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/* input(structure) - allows to make INSERT SELECT from incoming stream of data
|
||||
*/
|
||||
class TableFunctionInput : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "input";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
bool needStructureHint() const override { return true; }
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Input"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String structure;
|
||||
ColumnsDescription structure_hint;
|
||||
};
|
||||
|
||||
}
|
@ -2,7 +2,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/StorageMerge.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
@ -10,7 +9,6 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
@ -26,18 +24,46 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
[[noreturn]] void throwNoTablesMatchRegexp(const String & source_database_regexp, const String & source_table_regexp)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Error while executing table function merge. Either there is no database, which matches regular expression `{}`, or there are "
|
||||
"no tables in database matches `{}`, which fit tables expression: {}",
|
||||
source_database_regexp,
|
||||
source_database_regexp,
|
||||
source_table_regexp);
|
||||
}
|
||||
|
||||
[[noreturn]] void throwNoTablesMatchRegexp(const String & source_database_regexp, const String & source_table_regexp)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Error while executing table function merge. Either there is no database, which matches regular expression `{}`, or there are "
|
||||
"no tables in database matches `{}`, which fit tables expression: {}",
|
||||
source_database_regexp,
|
||||
source_database_regexp,
|
||||
source_table_regexp);
|
||||
}
|
||||
|
||||
/* merge (db_name, tables_regexp) - creates a temporary StorageMerge.
|
||||
* The structure of the table is taken from the first table that came up, suitable for regexp.
|
||||
* If there is no such table, an exception is thrown.
|
||||
*/
|
||||
class TableFunctionMerge : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "merge";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Merge"; }
|
||||
|
||||
using TableSet = std::set<String>;
|
||||
using DBToTableSetMap = std::map<String, TableSet>;
|
||||
const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const;
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context);
|
||||
|
||||
String source_database_name_or_regexp;
|
||||
String source_table_regexp;
|
||||
bool database_is_regexp = false;
|
||||
mutable std::optional<DBToTableSetMap> source_databases_and_tables;
|
||||
};
|
||||
|
||||
std::vector<size_t> TableFunctionMerge::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const
|
||||
{
|
||||
auto & table_function_node = query_node_table_function->as<TableFunctionNode &>();
|
||||
@ -179,6 +205,8 @@ TableFunctionMerge::getMatchedTablesWithAccess(const String & database_name, con
|
||||
return tables;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionMerge(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionMerge>();
|
||||
|
@ -1,38 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* merge (db_name, tables_regexp) - creates a temporary StorageMerge.
|
||||
* The structure of the table is taken from the first table that came up, suitable for regexp.
|
||||
* If there is no such table, an exception is thrown.
|
||||
*/
|
||||
class TableFunctionMerge : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "merge";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Merge"; }
|
||||
|
||||
using TableSet = std::set<String>;
|
||||
using DBToTableSetMap = std::map<String, TableSet>;
|
||||
const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const;
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context);
|
||||
|
||||
String source_database_name_or_regexp;
|
||||
String source_table_regexp;
|
||||
bool database_is_regexp = false;
|
||||
mutable std::optional<DBToTableSetMap> source_databases_and_tables;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -1,12 +1,11 @@
|
||||
#include <TableFunctions/TableFunctionMongoDB.h>
|
||||
#include <Storages/StorageMongoDB.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
@ -25,6 +24,29 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class TableFunctionMongoDB : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mongodb";
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "MongoDB"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
std::optional<StorageMongoDB::Configuration> configuration;
|
||||
String structure;
|
||||
};
|
||||
|
||||
StoragePtr TableFunctionMongoDB::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const
|
||||
@ -97,6 +119,7 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt
|
||||
configuration = StorageMongoDB::getConfiguration(main_arguments, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionMongoDB(TableFunctionFactory & factory)
|
||||
{
|
||||
|
@ -1,31 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/StorageMongoDB.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableFunctionMongoDB : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mongodb";
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "MongoDB"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
std::optional<StorageMongoDB::Configuration> configuration;
|
||||
String structure;
|
||||
};
|
||||
|
||||
}
|
@ -1,15 +1,15 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
|
||||
#include <Storages/StorageMySQL.h>
|
||||
#include <Processors/Sources/MySQLSource.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/MySQL/MySQLSettings.h>
|
||||
#include <Storages/MySQL/MySQLHelpers.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionMySQL.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <Common/quoteString.h>
|
||||
@ -27,6 +27,32 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* mysql ('host:port', database, table, user, password) - creates a temporary StorageMySQL.
|
||||
* The structure of the table is taken from the mysql query DESCRIBE table.
|
||||
* If there is no such table, an exception is thrown.
|
||||
*/
|
||||
class TableFunctionMySQL : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mysql";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "MySQL"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
mutable std::optional<mysqlxx::PoolWithFailover> pool;
|
||||
std::optional<StorageMySQL::Configuration> configuration;
|
||||
};
|
||||
|
||||
void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
const auto & args_func = ast_function->as<ASTFunction &>();
|
||||
@ -88,11 +114,14 @@ StoragePtr TableFunctionMySQL::executeImpl(
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionMySQL(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionMySQL>();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,38 +0,0 @@
|
||||
#pragma once
|
||||
#include "config.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/StorageMySQL.h>
|
||||
#include <mysqlxx/Pool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* mysql ('host:port', database, table, user, password) - creates a temporary StorageMySQL.
|
||||
* The structure of the table is taken from the mysql query DESCRIBE table.
|
||||
* If there is no such table, an exception is thrown.
|
||||
*/
|
||||
class TableFunctionMySQL : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mysql";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "MySQL"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
mutable std::optional<mysqlxx::PoolWithFailover> pool;
|
||||
std::optional<StorageMySQL::Configuration> configuration;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,12 +1,12 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Storages/StorageNull.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionNull.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
|
||||
@ -17,6 +17,36 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* null(structure) - creates a temporary null storage
|
||||
*
|
||||
* Used for testing purposes, for convenience writing tests and demos.
|
||||
*/
|
||||
class TableFunctionNull : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "null";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool needStructureHint() const override { return structure == "auto"; }
|
||||
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
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 "Null"; }
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
String structure = "auto";
|
||||
ColumnsDescription structure_hint;
|
||||
|
||||
const ColumnsDescription default_structure{NamesAndTypesList{{"dummy", std::make_shared<DataTypeUInt8>()}}};
|
||||
};
|
||||
|
||||
void TableFunctionNull::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
const auto * function = ast_function->as<ASTFunction>();
|
||||
@ -54,8 +84,11 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, Conte
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionNull(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionNull>({.documentation = {}, .allow_readonly = true});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,37 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* null(structure) - creates a temporary null storage
|
||||
*
|
||||
* Used for testing purposes, for convenience writing tests and demos.
|
||||
*/
|
||||
class TableFunctionNull : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "null";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
bool needStructureHint() const override { return structure == "auto"; }
|
||||
|
||||
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
|
||||
|
||||
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 "Null"; }
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
String structure = "auto";
|
||||
ColumnsDescription structure_hint;
|
||||
|
||||
const ColumnsDescription default_structure{NamesAndTypesList{{"dummy", std::make_shared<DataTypeUInt8>()}}};
|
||||
};
|
||||
}
|
@ -1,5 +1,4 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionNumbers.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -21,6 +20,28 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* numbers(limit), numbers_mt(limit)
|
||||
* - the same as SELECT number FROM system.numbers LIMIT limit.
|
||||
* Used for testing purposes, as a simple example of table function.
|
||||
*/
|
||||
template <bool multithreaded>
|
||||
class TableFunctionNumbers : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = multithreaded ? "numbers_mt" : "numbers";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "SystemNumbers"; }
|
||||
|
||||
UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
};
|
||||
|
||||
template <bool multithreaded>
|
||||
ColumnsDescription TableFunctionNumbers<multithreaded>::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const
|
||||
@ -49,12 +70,6 @@ StoragePtr TableFunctionNumbers<multithreaded>::executeImpl(const ASTPtr & ast_f
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'limit' or 'offset, limit'.", getName());
|
||||
}
|
||||
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionNumbers<true>>({.documentation = {}, .allow_readonly = true});
|
||||
factory.registerFunction<TableFunctionNumbers<false>>({.documentation = {}, .allow_readonly = true});
|
||||
}
|
||||
|
||||
template <bool multithreaded>
|
||||
UInt64 TableFunctionNumbers<multithreaded>::evaluateArgument(ContextPtr context, ASTPtr & argument) const
|
||||
{
|
||||
@ -72,3 +87,11 @@ UInt64 TableFunctionNumbers<multithreaded>::evaluateArgument(ContextPtr context,
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionNumbers<true>>({.documentation = {}, .allow_readonly = true});
|
||||
factory.registerFunction<TableFunctionNumbers<false>>({.documentation = {}, .allow_readonly = true});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,31 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* numbers(limit), numbers_mt(limit)
|
||||
* - the same as SELECT number FROM system.numbers LIMIT limit.
|
||||
* Used for testing purposes, as a simple example of table function.
|
||||
*/
|
||||
template <bool multithreaded>
|
||||
class TableFunctionNumbers : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = multithreaded ? "numbers_mt" : "numbers";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "SystemNumbers"; }
|
||||
|
||||
UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -1,13 +1,15 @@
|
||||
#include <TableFunctions/TableFunctionPostgreSQL.h>
|
||||
#include "config.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/Exception.h>
|
||||
#include "registerTableFunctions.h"
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -18,6 +20,28 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class TableFunctionPostgreSQL : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "postgresql";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "PostgreSQL"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
postgres::PoolWithFailoverPtr connection_pool;
|
||||
std::optional<StoragePostgreSQL::Configuration> configuration;
|
||||
};
|
||||
|
||||
StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const
|
||||
@ -60,6 +84,8 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex
|
||||
settings.postgresql_connection_pool_auto_close_connection);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionPostgreSQL(TableFunctionFactory & factory)
|
||||
{
|
||||
|
@ -1,36 +0,0 @@
|
||||
#pragma once
|
||||
#include "config.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableFunctionPostgreSQL : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "postgresql";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "PostgreSQL"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
postgres::PoolWithFailoverPtr connection_pool;
|
||||
std::optional<StoragePostgreSQL::Configuration> configuration;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,5 +1,3 @@
|
||||
#include <TableFunctions/TableFunctionRedis.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/parseAddress.h>
|
||||
|
||||
@ -15,6 +13,10 @@
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <Storages/StorageRedis.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,6 +26,33 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* Implements Redis table function.
|
||||
* Use redis(host:port, key, structure[, db_index[, password[, pool_size]]]);
|
||||
*/
|
||||
class TableFunctionRedis : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "redis";
|
||||
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 "Redis"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
RedisConfiguration configuration;
|
||||
String structure;
|
||||
String primary_key;
|
||||
};
|
||||
|
||||
StoragePtr TableFunctionRedis::executeImpl(
|
||||
const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const
|
||||
{
|
||||
@ -85,6 +114,7 @@ void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad arguments redis table function structure should contains key.");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionRedis(TableFunctionFactory & factory)
|
||||
{
|
||||
|
@ -1,34 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/StorageRedis.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* Implements Redis table function.
|
||||
* Use redis(host:port, key, structure[, db_index[, password[, pool_size]]]);
|
||||
*/
|
||||
class TableFunctionRedis : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "redis";
|
||||
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 "Redis"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
RedisConfiguration configuration;
|
||||
String structure;
|
||||
String primary_key;
|
||||
};
|
||||
|
||||
}
|
@ -1,9 +1,10 @@
|
||||
#include <TableFunctions/TableFunctionSQLite.h>
|
||||
#include "config.h"
|
||||
|
||||
#if USE_SQLITE
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/StorageSQLite.h>
|
||||
|
||||
#include <Databases/SQLite/SQLiteUtils.h>
|
||||
#include "registerTableFunctions.h"
|
||||
@ -12,7 +13,6 @@
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
@ -27,6 +27,28 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class TableFunctionSQLite : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "sqlite";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "SQLite"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String database_path, remote_table_name;
|
||||
std::shared_ptr<sqlite3> sqlite_db;
|
||||
};
|
||||
|
||||
StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const
|
||||
@ -69,6 +91,7 @@ void TableFunctionSQLite::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
sqlite_db = openSQLiteDB(database_path, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionSQLite(TableFunctionFactory & factory)
|
||||
{
|
||||
|
@ -1,34 +0,0 @@
|
||||
#pragma once
|
||||
#include "config.h"
|
||||
|
||||
#if USE_SQLITE
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/StorageSQLite.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableFunctionSQLite : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "sqlite";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "SQLite"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String database_path, remote_table_name;
|
||||
std::shared_ptr<sqlite3> sqlite_db;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -9,7 +9,6 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionValues.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
|
||||
@ -30,7 +29,32 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
||||
}
|
||||
|
||||
static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args, const Block & sample_block, size_t start, ContextPtr context)
|
||||
namespace
|
||||
{
|
||||
|
||||
/* values(structure, values...) - creates a temporary storage filling columns with values
|
||||
* values is case-insensitive table function.
|
||||
*/
|
||||
class TableFunctionValues : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "values";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Values"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
static DataTypes getTypesFromArgument(const ASTPtr & arg, ContextPtr context);
|
||||
|
||||
ColumnsDescription structure;
|
||||
bool has_structure_in_arguments;
|
||||
};
|
||||
|
||||
void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args, const Block & sample_block, size_t start, ContextPtr context)
|
||||
{
|
||||
if (res_columns.size() == 1) /// Parsing arguments as Fields
|
||||
{
|
||||
@ -146,6 +170,8 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, Context
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionValues(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionValues>({.documentation = {}, .allow_readonly = true}, TableFunctionFactory::CaseInsensitive);
|
||||
|
@ -1,30 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/* values(structure, values...) - creates a temporary storage filling columns with values
|
||||
* values is case-insensitive table function.
|
||||
*/
|
||||
class TableFunctionValues : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "values";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "Values"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
static DataTypes getTypesFromArgument(const ASTPtr & arg, ContextPtr context);
|
||||
|
||||
ColumnsDescription structure;
|
||||
bool has_structure_in_arguments;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -3,12 +3,13 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <base/types.h>
|
||||
#include <Storages/StorageNull.h>
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionViewIfPermitted.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include "registerTableFunctions.h"
|
||||
@ -16,6 +17,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
@ -23,11 +25,37 @@ namespace ErrorCodes
|
||||
extern const int ACCESS_DENIED;
|
||||
}
|
||||
|
||||
|
||||
const ASTSelectWithUnionQuery & TableFunctionViewIfPermitted::getSelectQuery() const
|
||||
namespace
|
||||
{
|
||||
return *create.select;
|
||||
}
|
||||
|
||||
/* viewIfPermitted(query ELSE null('structure'))
|
||||
* Works as "view(query)" if the current user has the permissions required to execute "query"; works as "null('structure')" otherwise.
|
||||
*/
|
||||
class TableFunctionViewIfPermitted : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "viewIfPermitted";
|
||||
|
||||
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 "ViewIfPermitted"; }
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
bool isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const;
|
||||
|
||||
ASTCreateQuery create;
|
||||
ASTPtr else_ast;
|
||||
TableFunctionPtr else_table_function;
|
||||
};
|
||||
|
||||
|
||||
std::vector<size_t> TableFunctionViewIfPermitted::skipAnalysisForArguments(const QueryTreeNodePtr &, ContextPtr) const
|
||||
{
|
||||
@ -118,6 +146,8 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionViewIfPermitted>({.documentation = {}, .allow_readonly = true});
|
||||
|
@ -1,40 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <base/types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* viewIfPermitted(query ELSE null('structure'))
|
||||
* Works as "view(query)" if the current user has the permissions required to execute "query"; works as "null('structure')" otherwise.
|
||||
*/
|
||||
class TableFunctionViewIfPermitted : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "viewIfPermitted";
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
const ASTSelectWithUnionQuery & getSelectQuery() const;
|
||||
|
||||
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 "ViewIfPermitted"; }
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
|
||||
bool isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const;
|
||||
|
||||
ASTCreateQuery create;
|
||||
ASTPtr else_ast;
|
||||
TableFunctionPtr else_table_function;
|
||||
};
|
||||
|
||||
}
|
@ -1,5 +1,4 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionZeros.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
@ -15,9 +14,32 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/* zeros(limit), zeros_mt(limit)
|
||||
* - the same as SELECT zero FROM system.zeros LIMIT limit.
|
||||
* Used for testing purposes, as a simple example of table function.
|
||||
*/
|
||||
template <bool multithreaded>
|
||||
class TableFunctionZeros : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = multithreaded ? "zeros_mt" : "zeros";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "SystemZeros"; }
|
||||
|
||||
UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
};
|
||||
|
||||
|
||||
template <bool multithreaded>
|
||||
ColumnsDescription TableFunctionZeros<multithreaded>::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const
|
||||
@ -46,6 +68,14 @@ StoragePtr TableFunctionZeros<multithreaded>::executeImpl(const ASTPtr & ast_fun
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'limit'.", getName());
|
||||
}
|
||||
|
||||
template <bool multithreaded>
|
||||
UInt64 TableFunctionZeros<multithreaded>::evaluateArgument(ContextPtr context, ASTPtr & argument) const
|
||||
{
|
||||
return checkAndGetLiteralArgument<UInt64>(evaluateConstantExpressionOrIdentifierAsLiteral(argument, context), "length");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionZeros(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionZeros<true>>({.documentation = {
|
||||
@ -71,13 +101,7 @@ void registerTableFunctionZeros(TableFunctionFactory & factory)
|
||||
See also the `system.zeros` table.
|
||||
)",
|
||||
.examples={{"1", "SELECT count() FROM zeros_mt(1000000000) WHERE NOT ignore(randomPrintableASCII(10))", ""}}
|
||||
}});
|
||||
}
|
||||
|
||||
template <bool multithreaded>
|
||||
UInt64 TableFunctionZeros<multithreaded>::evaluateArgument(ContextPtr context, ASTPtr & argument) const
|
||||
{
|
||||
return checkAndGetLiteralArgument<UInt64>(evaluateConstantExpressionOrIdentifierAsLiteral(argument, context), "length");
|
||||
}});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,31 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* zeros(limit), zeros_mt(limit)
|
||||
* - the same as SELECT zero FROM system.zeros LIMIT limit.
|
||||
* Used for testing purposes, as a simple example of table function.
|
||||
*/
|
||||
template <bool multithreaded>
|
||||
class TableFunctionZeros : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = multithreaded ? "zeros_mt" : "zeros";
|
||||
std::string getName() const override { return name; }
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
|
||||
const char * getStorageTypeName() const override { return "SystemZeros"; }
|
||||
|
||||
UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
|
||||
};
|
||||
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user