mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Better exception messages on wrong table engines/functions argument types
This commit is contained in:
parent
b2776e3dcd
commit
59c1c472cb
@ -559,4 +559,31 @@ String toString(const Field & x)
|
||||
x);
|
||||
}
|
||||
|
||||
String fieldTypeToString(Field::Types::Which type)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case Field::Types::Which::Null: return "Null";
|
||||
case Field::Types::Which::Array: return "Array";
|
||||
case Field::Types::Which::Tuple: return "Tuple";
|
||||
case Field::Types::Which::Map: return "Map";
|
||||
case Field::Types::Which::Object: return "Object";
|
||||
case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState";
|
||||
case Field::Types::Which::Bool: return "Bool";
|
||||
case Field::Types::Which::String: return "String";
|
||||
case Field::Types::Which::Decimal32: return "Decimal32";
|
||||
case Field::Types::Which::Decimal64: return "Decimal64";
|
||||
case Field::Types::Which::Decimal128: return "Decimal128";
|
||||
case Field::Types::Which::Decimal256: return "Decimal256";
|
||||
case Field::Types::Which::Float64: return "Float64";
|
||||
case Field::Types::Which::Int64: return "Int64";
|
||||
case Field::Types::Which::Int128: return "Int128";
|
||||
case Field::Types::Which::Int256: return "Int256";
|
||||
case Field::Types::Which::UInt64: return "UInt64";
|
||||
case Field::Types::Which::UInt128: return "UInt128";
|
||||
case Field::Types::Which::UInt256: return "UInt256";
|
||||
case Field::Types::Which::UUID: return "UUID";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1011,6 +1011,8 @@ void writeFieldText(const Field & x, WriteBuffer & buf);
|
||||
|
||||
String toString(const Field & x);
|
||||
|
||||
String fieldTypeToString(Field::Types::Which type);
|
||||
|
||||
}
|
||||
|
||||
template <>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Interpreters/getClusterName.h>
|
||||
@ -22,7 +23,7 @@ std::string getClusterName(const IAST & node)
|
||||
return ast_id->name();
|
||||
|
||||
if (const auto * ast_lit = node.as<ASTLiteral>())
|
||||
return ast_lit->value.safeGet<String>();
|
||||
return checkAndGetLiteralArgument<String>(*ast_lit, "cluster_name");
|
||||
|
||||
/// A hack to support hyphens in cluster names.
|
||||
if (const auto * ast_func = node.as<ASTFunction>())
|
||||
|
@ -9,16 +9,14 @@
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Storages/FileLog/FileLogSource.h>
|
||||
#include <Storages/FileLog/ReadBufferFromFileLog.h>
|
||||
#include <Storages/FileLog/StorageFileLog.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/Macros.h>
|
||||
@ -805,8 +803,8 @@ void registerStorageFileLog(StorageFactory & factory)
|
||||
auto path_ast = evaluateConstantExpressionAsLiteral(engine_args[0], args.getContext());
|
||||
auto format_ast = evaluateConstantExpressionAsLiteral(engine_args[1], args.getContext());
|
||||
|
||||
auto path = path_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto format = format_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto path = checkAndGetLiteralArgument<String>(path_ast, "path");
|
||||
auto format = checkAndGetLiteralArgument<String>(format_ast, "format");
|
||||
|
||||
return std::make_shared<StorageFileLog>(
|
||||
args.table_id,
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/PartitionedSink.h>
|
||||
#include <Storages/getVirtualsForStorage.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
@ -664,13 +665,13 @@ void registerStorageHDFS(StorageFactory & factory)
|
||||
|
||||
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext());
|
||||
|
||||
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String url = checkAndGetLiteralArgument<String>(engine_args[0], "url");
|
||||
|
||||
String format_name = "auto";
|
||||
if (engine_args.size() > 1)
|
||||
{
|
||||
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext());
|
||||
format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
format_name = checkAndGetLiteralArgument<String>(engine_args[1], "format_name");
|
||||
}
|
||||
|
||||
if (format_name == "auto")
|
||||
@ -680,7 +681,7 @@ void registerStorageHDFS(StorageFactory & factory)
|
||||
if (engine_args.size() == 3)
|
||||
{
|
||||
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext());
|
||||
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
compression_method = checkAndGetLiteralArgument<String>(engine_args[2], "compression_method");
|
||||
} else compression_method = "auto";
|
||||
|
||||
ASTPtr partition_by;
|
||||
|
@ -37,6 +37,7 @@
|
||||
#include <Storages/Hive/StorageHiveMetadata.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -938,9 +939,9 @@ void registerStorageHive(StorageFactory & factory)
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
|
||||
|
||||
const String & hive_metastore_url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const String & hive_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const String & hive_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const String & hive_metastore_url = checkAndGetLiteralArgument<String>(engine_args[0], "hive_metastore_url");
|
||||
const String & hive_database = checkAndGetLiteralArgument<String>(engine_args[1], "hive_database");
|
||||
const String & hive_table = checkAndGetLiteralArgument<String>(engine_args[2], "hive_table");
|
||||
return std::make_shared<StorageHive>(
|
||||
hive_metastore_url,
|
||||
hive_database,
|
||||
|
@ -1,11 +1,9 @@
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <memory>
|
||||
#include <Core/Types.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
@ -14,7 +13,7 @@
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/parseAddress.h>
|
||||
|
||||
@ -156,11 +155,11 @@ MeiliSearchConfiguration StorageMeiliSearch::getConfiguration(ASTs engine_args,
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
||||
|
||||
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String index = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String url = checkAndGetLiteralArgument<String>(engine_args[0], "url");
|
||||
String index = checkAndGetLiteralArgument<String>(engine_args[1], "index");
|
||||
String key;
|
||||
if (engine_args.size() == 3)
|
||||
key = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
key = checkAndGetLiteralArgument<String>(engine_args[2], "key");
|
||||
return MeiliSearchConfiguration(url, index, key);
|
||||
}
|
||||
}
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Storages/StorageBuffer.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -1079,8 +1080,8 @@ void registerStorageBuffer(StorageFactory & factory)
|
||||
|
||||
size_t i = 0;
|
||||
|
||||
String destination_database = engine_args[i++]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String destination_table = engine_args[i++]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String destination_database = checkAndGetLiteralArgument<String>(engine_args[i++], "destination_database");
|
||||
String destination_table = checkAndGetLiteralArgument<String>(engine_args[i++], "destination_table");
|
||||
|
||||
UInt64 num_buckets = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), engine_args[i++]->as<ASTLiteral &>().value);
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -339,7 +340,7 @@ void registerStorageDictionary(StorageFactory & factory)
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
args.engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[0], local_context);
|
||||
String dictionary_name = args.engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String dictionary_name = checkAndGetLiteralArgument<String>(args.engine_args[0], "dictionary_name");
|
||||
|
||||
if (!args.attach)
|
||||
{
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/getStructureOfRemoteTable.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
@ -34,10 +35,6 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
@ -1437,15 +1434,15 @@ void registerStorageDistributed(StorageFactory & factory)
|
||||
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], local_context);
|
||||
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], local_context);
|
||||
|
||||
String remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String remote_database = checkAndGetLiteralArgument<String>(engine_args[1], "remote_database");
|
||||
String remote_table = checkAndGetLiteralArgument<String>(engine_args[2], "remote_table");
|
||||
|
||||
const auto & sharding_key = engine_args.size() >= 4 ? engine_args[3] : nullptr;
|
||||
String storage_policy = "default";
|
||||
if (engine_args.size() >= 5)
|
||||
{
|
||||
engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], local_context);
|
||||
storage_policy = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
storage_policy = checkAndGetLiteralArgument<String>(engine_args[4], "storage_policy");
|
||||
}
|
||||
|
||||
/// Check that sharding_key exists in the table and has numeric type.
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -179,14 +180,14 @@ void registerStorageExecutable(StorageFactory & factory)
|
||||
for (size_t i = 0; i < 2; ++i)
|
||||
args.engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[i], local_context);
|
||||
|
||||
auto scipt_name_with_arguments_value = args.engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto script_name_with_arguments_value = checkAndGetLiteralArgument<String>(args.engine_args[0], "script_name_with_arguments_value");
|
||||
|
||||
std::vector<String> script_name_with_arguments;
|
||||
boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c) { return c == ' '; });
|
||||
boost::split(script_name_with_arguments, script_name_with_arguments_value, [](char c) { return c == ' '; });
|
||||
|
||||
auto script_name = script_name_with_arguments[0];
|
||||
script_name_with_arguments.erase(script_name_with_arguments.begin());
|
||||
auto format = args.engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto format = checkAndGetLiteralArgument<String>(args.engine_args[1], "format");
|
||||
|
||||
std::vector<ASTPtr> input_queries;
|
||||
for (size_t i = 2; i < args.engine_args.size(); ++i)
|
||||
|
@ -3,13 +3,9 @@
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <Storages/StorageMySQL.h>
|
||||
@ -17,6 +13,7 @@
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
#include <Storages/StorageURL.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
|
||||
@ -229,7 +226,7 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
if (engine_args.size() < 2)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine ExternalDistributed must have at least 2 arguments: engine_name, named_collection and/or description");
|
||||
|
||||
auto engine_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto engine_name = checkAndGetLiteralArgument<String>(engine_args[0], "engine_name");
|
||||
StorageExternalDistributed::ExternalStorageEngine table_engine;
|
||||
if (engine_name == "URL")
|
||||
table_engine = StorageExternalDistributed::ExternalStorageEngine::URL;
|
||||
@ -256,7 +253,7 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
for (const auto & [name, value] : storage_specific_args)
|
||||
{
|
||||
if (name == "description")
|
||||
cluster_description = value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
cluster_description = checkAndGetLiteralArgument<String>(value, "cluster_description");
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unknown key-value argument {} for table engine URL", name);
|
||||
@ -271,11 +268,11 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
|
||||
|
||||
cluster_description = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.format = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
cluster_description = checkAndGetLiteralArgument<String>(engine_args[1], "cluster_description");
|
||||
configuration.format = checkAndGetLiteralArgument<String>(engine_args[2], "format");
|
||||
configuration.compression_method = "auto";
|
||||
if (engine_args.size() == 4)
|
||||
configuration.compression_method = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.compression_method = checkAndGetLiteralArgument<String>(engine_args[3], "compression_method");
|
||||
}
|
||||
|
||||
|
||||
@ -302,7 +299,7 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
for (const auto & [name, value] : storage_specific_args)
|
||||
{
|
||||
if (name == "description")
|
||||
cluster_description = value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
cluster_description = checkAndGetLiteralArgument<String>(value, "cluster_description");
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unknown key-value argument {} for table function URL", name);
|
||||
@ -320,11 +317,11 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
"ExternalDistributed('engine_name', 'cluster_description', 'database', 'table', 'user', 'password').",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
cluster_description = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.database = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.table = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.username = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.password = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
cluster_description = checkAndGetLiteralArgument<String>(engine_args[1], "cluster_description");
|
||||
configuration.database = checkAndGetLiteralArgument<String>(engine_args[2], "database");
|
||||
configuration.table = checkAndGetLiteralArgument<String>(engine_args[3], "table");
|
||||
configuration.username = checkAndGetLiteralArgument<String>(engine_args[4], "username");
|
||||
configuration.password = checkAndGetLiteralArgument<String>(engine_args[5], "password");
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,5 +1,10 @@
|
||||
#include <Storages/StorageFile.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/PartitionedSink.h>
|
||||
#include <Storages/Distributed/DirectoryMonitor.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -20,30 +25,26 @@
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/parseGlobs.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/PartitionedSink.h>
|
||||
|
||||
#include <sys/stat.h>
|
||||
#include <fcntl.h>
|
||||
#include <unistd.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
#include <filesystem>
|
||||
#include <Storages/Distributed/DirectoryMonitor.h>
|
||||
#include <Processors/ISource.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/parseGlobs.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
|
||||
#include <sys/stat.h>
|
||||
#include <fcntl.h>
|
||||
#include <unistd.h>
|
||||
#include <re2/re2.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
@ -1103,7 +1104,7 @@ void registerStorageFile(StorageFactory & factory)
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.getLocalContext());
|
||||
storage_args.format_name = engine_args_ast[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
storage_args.format_name = checkAndGetLiteralArgument<String>(engine_args_ast[0], "format_name");
|
||||
|
||||
// Use format settings from global server context + settings from
|
||||
// the SETTINGS clause of the create query. Settings from current
|
||||
@ -1171,7 +1172,7 @@ void registerStorageFile(StorageFactory & factory)
|
||||
if (engine_args_ast.size() == 3)
|
||||
{
|
||||
engine_args_ast[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[2], factory_args.getLocalContext());
|
||||
storage_args.compression_method = engine_args_ast[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
storage_args.compression_method = checkAndGetLiteralArgument<String>(engine_args_ast[2], "compression_method");
|
||||
}
|
||||
else
|
||||
storage_args.compression_method = "auto";
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageGenerateRandom.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -12,7 +13,6 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
@ -469,16 +469,16 @@ void registerStorageGenerateRandom(StorageFactory & factory)
|
||||
|
||||
if (!engine_args.empty())
|
||||
{
|
||||
const Field & value = engine_args[0]->as<const ASTLiteral &>().value;
|
||||
if (!value.isNull())
|
||||
random_seed = value.safeGet<UInt64>();
|
||||
const auto & ast_literal = engine_args[0]->as<const ASTLiteral &>();
|
||||
if (!ast_literal.value.isNull())
|
||||
random_seed = checkAndGetLiteralArgument<UInt64>(ast_literal, "random_seed");
|
||||
}
|
||||
|
||||
if (engine_args.size() >= 2)
|
||||
max_string_length = engine_args[1]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||
max_string_length = checkAndGetLiteralArgument<UInt64>(engine_args[1], "max_string_length");
|
||||
|
||||
if (engine_args.size() == 3)
|
||||
max_array_length = engine_args[2]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||
max_array_length = checkAndGetLiteralArgument<UInt64>(engine_args[2], "max_array_length");
|
||||
|
||||
return std::make_shared<StorageGenerateRandom>(args.table_id, args.columns, args.comment, max_array_length, max_string_length, random_seed);
|
||||
});
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <QueryPipeline/narrowPipe.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/StorageMerge.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
@ -22,17 +23,16 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include "Processors/QueryPlan/BuildQueryPipelineSettings.h"
|
||||
#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h"
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <base/range.h>
|
||||
#include <algorithm>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/Transforms/MaterializingTransform.h>
|
||||
#include <Processors/ConcatProcessor.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <base/range.h>
|
||||
#include <algorithm>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -848,7 +848,7 @@ std::tuple<bool /* is_regexp */, ASTPtr> StorageMerge::evaluateDatabaseName(cons
|
||||
throw Exception("REGEXP in Merge ENGINE takes only one argument", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto * literal = func->arguments->children[0]->as<ASTLiteral>();
|
||||
if (!literal || literal->value.safeGet<String>().empty())
|
||||
if (!literal || literal->value.getType() != Field::Types::Which::String || literal->value.safeGet<String>().empty())
|
||||
throw Exception("Argument for REGEXP in Merge ENGINE should be a non empty String Literal", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return {true, func->arguments->children[0]};
|
||||
@ -879,10 +879,10 @@ void registerStorageMerge(StorageFactory & factory)
|
||||
if (!is_regexp)
|
||||
engine_args[0] = database_ast;
|
||||
|
||||
String source_database_name_or_regexp = database_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String source_database_name_or_regexp = checkAndGetLiteralArgument<String>(database_ast, "database_name");
|
||||
|
||||
engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.getLocalContext());
|
||||
String table_name_regexp = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String table_name_regexp = checkAndGetLiteralArgument<String>(engine_args[1], "table_name_regexp");
|
||||
|
||||
return std::make_shared<StorageMerge>(
|
||||
args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, table_name_regexp, args.getContext());
|
||||
|
@ -1,11 +1,12 @@
|
||||
#include "StorageMongoDB.h"
|
||||
#include "StorageMongoDBSocketFactory.h"
|
||||
#include <Storages/StorageMongoDB.h>
|
||||
#include <Storages/StorageMongoDBSocketFactory.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
#include <Poco/MongoDB/Cursor.h>
|
||||
#include <Poco/MongoDB/Database.h>
|
||||
#include <Poco/Version.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -120,7 +121,7 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C
|
||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
||||
{
|
||||
if (arg_name == "options")
|
||||
configuration.options = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.options = checkAndGetLiteralArgument<String>(arg_value, "options");
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key-value argument."
|
||||
@ -139,17 +140,17 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
||||
|
||||
/// 27017 is the default MongoDB port.
|
||||
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 27017);
|
||||
auto parsed_host_port = parseAddress(checkAndGetLiteralArgument<String>(engine_args[0], "host:port"), 27017);
|
||||
|
||||
configuration.host = parsed_host_port.first;
|
||||
configuration.port = parsed_host_port.second;
|
||||
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.database = checkAndGetLiteralArgument<String>(engine_args[1], "database");
|
||||
configuration.table = checkAndGetLiteralArgument<String>(engine_args[2], "table");
|
||||
configuration.username = checkAndGetLiteralArgument<String>(engine_args[3], "username");
|
||||
configuration.password = checkAndGetLiteralArgument<String>(engine_args[4], "password");
|
||||
|
||||
if (engine_args.size() >= 6)
|
||||
configuration.options = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.options = checkAndGetLiteralArgument<String>(engine_args[5], "database");
|
||||
|
||||
}
|
||||
|
||||
|
@ -5,14 +5,12 @@
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Storages/MySQL/MySQLHelpers.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Processors/Sources/MySQLSource.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -253,9 +251,9 @@ StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, Conte
|
||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
||||
{
|
||||
if (arg_name == "replace_query")
|
||||
configuration.replace_query = arg_value->as<ASTLiteral>()->value.safeGet<bool>();
|
||||
configuration.replace_query = checkAndGetLiteralArgument<bool>(arg_value, "replace_query");
|
||||
else if (arg_name == "on_duplicate_clause")
|
||||
configuration.on_duplicate_clause = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.on_duplicate_clause = checkAndGetLiteralArgument<String>(arg_value, "on_duplicate_clause");
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key-value argument."
|
||||
@ -273,18 +271,18 @@ StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, Conte
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context_);
|
||||
|
||||
const auto & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const auto & host_port = checkAndGetLiteralArgument<String>(engine_args[0], "host:port");
|
||||
size_t max_addresses = context_->getSettingsRef().glob_expansion_max_elements;
|
||||
|
||||
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
|
||||
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.database = checkAndGetLiteralArgument<String>(engine_args[1], "database");
|
||||
configuration.table = checkAndGetLiteralArgument<String>(engine_args[2], "table");
|
||||
configuration.username = checkAndGetLiteralArgument<String>(engine_args[3], "username");
|
||||
configuration.password = checkAndGetLiteralArgument<String>(engine_args[4], "password");
|
||||
if (engine_args.size() >= 6)
|
||||
configuration.replace_query = engine_args[5]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
||||
configuration.replace_query = checkAndGetLiteralArgument<UInt64>(engine_args[5], "replace_query");
|
||||
if (engine_args.size() == 7)
|
||||
configuration.on_duplicate_clause = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.on_duplicate_clause = checkAndGetLiteralArgument<String>(engine_args[6], "on_duplicate_clause");
|
||||
}
|
||||
for (const auto & address : configuration.addresses)
|
||||
context_->getRemoteHostFilter().checkHostAndPort(address.first, toString(address.second));
|
||||
|
@ -11,8 +11,6 @@
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
@ -31,7 +29,6 @@
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Parsers/getInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
@ -40,6 +37,7 @@
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -400,7 +398,7 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a
|
||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
||||
{
|
||||
if (arg_name == "on_conflict")
|
||||
configuration.on_conflict = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.on_conflict = checkAndGetLiteralArgument<String>(arg_value, "on_conflict");
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key-value argument."
|
||||
@ -418,7 +416,7 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
||||
|
||||
const auto & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const auto & host_port = checkAndGetLiteralArgument<String>(engine_args[0], "host:port");
|
||||
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
||||
|
||||
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
|
||||
@ -427,15 +425,15 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a
|
||||
configuration.host = configuration.addresses[0].first;
|
||||
configuration.port = configuration.addresses[0].second;
|
||||
}
|
||||
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.database = checkAndGetLiteralArgument<String>(engine_args[1], "host:port");
|
||||
configuration.table = checkAndGetLiteralArgument<String>(engine_args[2], "table");
|
||||
configuration.username = checkAndGetLiteralArgument<String>(engine_args[3], "username");
|
||||
configuration.password = checkAndGetLiteralArgument<String>(engine_args[4], "password");
|
||||
|
||||
if (engine_args.size() >= 6)
|
||||
configuration.schema = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.schema = checkAndGetLiteralArgument<String>(engine_args[5], "schema");
|
||||
if (engine_args.size() >= 7)
|
||||
configuration.on_conflict = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.on_conflict = checkAndGetLiteralArgument<String>(engine_args[6], "on_conflict");
|
||||
}
|
||||
for (const auto & address : configuration.addresses)
|
||||
context->getRemoteHostFilter().checkHostAndPort(address.first, toString(address.second));
|
||||
|
@ -11,14 +11,12 @@
|
||||
|
||||
#include <IO/S3Common.h>
|
||||
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
@ -27,6 +25,7 @@
|
||||
#include <Storages/PartitionedSink.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/getVirtualsForStorage.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
@ -1051,25 +1050,25 @@ void StorageS3::processNamedCollectionResult(StorageS3Configuration & configurat
|
||||
for (const auto & [arg_name, arg_value] : key_value_args)
|
||||
{
|
||||
if (arg_name == "access_key_id")
|
||||
configuration.auth_settings.access_key_id = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.auth_settings.access_key_id = checkAndGetLiteralArgument<String>(arg_value, "access_key_id");
|
||||
else if (arg_name == "secret_access_key")
|
||||
configuration.auth_settings.secret_access_key = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument<String>(arg_value, "secret_access_key");
|
||||
else if (arg_name == "filename")
|
||||
configuration.url = std::filesystem::path(configuration.url) / arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.url = std::filesystem::path(configuration.url) / checkAndGetLiteralArgument<String>(arg_value, "filename");
|
||||
else if (arg_name == "use_environment_credentials")
|
||||
configuration.auth_settings.use_environment_credentials = arg_value->as<ASTLiteral>()->value.safeGet<UInt8>();
|
||||
configuration.auth_settings.use_environment_credentials = checkAndGetLiteralArgument<UInt8>(arg_value, "use_environment_credentials");
|
||||
else if (arg_name == "max_single_read_retries")
|
||||
configuration.rw_settings.max_single_read_retries = arg_value->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument<UInt64>(arg_value, "max_single_read_retries");
|
||||
else if (arg_name == "min_upload_part_size")
|
||||
configuration.rw_settings.max_single_read_retries = arg_value->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument<UInt64>(arg_value, "min_upload_part_size");
|
||||
else if (arg_name == "upload_part_size_multiply_factor")
|
||||
configuration.rw_settings.max_single_read_retries = arg_value->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument<UInt64>(arg_value, "upload_part_size_multiply_factor");
|
||||
else if (arg_name == "upload_part_size_multiply_parts_count_threshold")
|
||||
configuration.rw_settings.max_single_read_retries = arg_value->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument<UInt64>(arg_value, "upload_part_size_multiply_parts_count_threshold");
|
||||
else if (arg_name == "max_single_part_upload_size")
|
||||
configuration.rw_settings.max_single_read_retries = arg_value->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument<UInt64>(arg_value, "max_single_part_upload_size");
|
||||
else if (arg_name == "max_connections")
|
||||
configuration.rw_settings.max_single_read_retries = arg_value->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument<UInt64>(arg_value, "max_connections");
|
||||
else
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
|
||||
@ -1098,22 +1097,22 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context);
|
||||
|
||||
configuration.url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.url = checkAndGetLiteralArgument<String>(engine_args[0], "url");
|
||||
if (engine_args.size() >= 4)
|
||||
{
|
||||
configuration.auth_settings.access_key_id = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.auth_settings.secret_access_key = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.auth_settings.access_key_id = checkAndGetLiteralArgument<String>(engine_args[1], "access_key_id");
|
||||
configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument<String>(engine_args[2], "secret_access_key");
|
||||
}
|
||||
|
||||
if (engine_args.size() == 3 || engine_args.size() == 5)
|
||||
{
|
||||
configuration.compression_method = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.format = engine_args[engine_args.size() - 2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.compression_method = checkAndGetLiteralArgument<String>(engine_args.back(), "compression_method");
|
||||
configuration.format = checkAndGetLiteralArgument<String>(engine_args[engine_args.size() - 2], "format");
|
||||
}
|
||||
else if (engine_args.size() != 1)
|
||||
{
|
||||
configuration.compression_method = "auto";
|
||||
configuration.format = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.format = checkAndGetLiteralArgument<String>(engine_args.back(), "format");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,12 +1,10 @@
|
||||
#include "StorageSQLite.h"
|
||||
|
||||
#if USE_SQLITE
|
||||
#include <base/range.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Processors/Sources/SQLiteSource.h>
|
||||
#include <Databases/SQLite/SQLiteUtils.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <IO/Operators.h>
|
||||
@ -16,6 +14,7 @@
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
@ -168,8 +167,8 @@ void registerStorageSQLite(StorageFactory & factory)
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
|
||||
|
||||
const auto database_path = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const auto table_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
const auto database_path = checkAndGetLiteralArgument<String>(engine_args[0], "database_path");
|
||||
const auto table_name = checkAndGetLiteralArgument<String>(engine_args[1], "table_name");
|
||||
|
||||
auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */!args.attach);
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Storages/StorageSet.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
@ -11,7 +10,6 @@
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <filesystem>
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <sparsehash/dense_hash_map>
|
||||
#include <sparsehash/dense_hash_set>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,8 +1,6 @@
|
||||
#include <sys/stat.h>
|
||||
#include <sys/types.h>
|
||||
#include <cerrno>
|
||||
|
||||
#include <map>
|
||||
#include <optional>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -12,7 +10,6 @@
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedReadBufferFromFile.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
@ -21,11 +18,8 @@
|
||||
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageStripeLog.h>
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Storages/StorageURL.h>
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <Storages/PartitionedSink.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -21,17 +23,15 @@
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <Storages/PartitionedSink.h>
|
||||
#include "Common/ThreadStatus.h"
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include "IO/HTTPCommon.h"
|
||||
#include "IO/ReadWriteBufferFromHTTP.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/ISource.h>
|
||||
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
@ -960,11 +960,11 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex
|
||||
if (header_it != args.end())
|
||||
args.erase(header_it);
|
||||
|
||||
configuration.url = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.url = checkAndGetLiteralArgument<String>(args[0], "url");
|
||||
if (args.size() > 1)
|
||||
configuration.format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.format = checkAndGetLiteralArgument<String>(args[1], "format");
|
||||
if (args.size() == 3)
|
||||
configuration.compression_method = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.compression_method = checkAndGetLiteralArgument<String>(args[2], "compression_method");
|
||||
}
|
||||
|
||||
if (configuration.format == "auto")
|
||||
|
@ -1,16 +1,16 @@
|
||||
#include "StorageXDBC.h"
|
||||
#include <Storages/StorageXDBC.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageURL.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ConnectionTimeoutsContext.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageURL.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
@ -173,11 +173,11 @@ namespace
|
||||
|
||||
BridgeHelperPtr bridge_helper = std::make_shared<XDBCBridgeHelper<BridgeHelperMixin>>(args.getContext(),
|
||||
args.getContext()->getSettingsRef().http_receive_timeout.value,
|
||||
engine_args[0]->as<ASTLiteral &>().value.safeGet<String>());
|
||||
checkAndGetLiteralArgument<String>(engine_args[0], "connection_string"));
|
||||
return std::make_shared<StorageXDBC>(
|
||||
args.table_id,
|
||||
engine_args[1]->as<ASTLiteral &>().value.safeGet<String>(),
|
||||
engine_args[2]->as<ASTLiteral &>().value.safeGet<String>(),
|
||||
checkAndGetLiteralArgument<String>(engine_args[1], "database_name"),
|
||||
checkAndGetLiteralArgument<String>(engine_args[2], "table_name"),
|
||||
args.columns,
|
||||
args.comment,
|
||||
args.getContext(),
|
||||
|
37
src/Storages/checkAndGetLiteralArgument.cpp
Normal file
37
src/Storages/checkAndGetLiteralArgument.cpp
Normal file
@ -0,0 +1,37 @@
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name)
|
||||
{
|
||||
return checkAndGetLiteralArgument<T>(*arg->as<ASTLiteral>(), arg_name);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name)
|
||||
{
|
||||
T res;
|
||||
if (arg.value.tryGet(res))
|
||||
return res;
|
||||
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Argument '{}' must be a literal with type {}, got {}",
|
||||
arg_name,
|
||||
fieldTypeToString(Field::TypeToEnum<T>::value),
|
||||
fieldTypeToString(arg.value.getType()));
|
||||
}
|
||||
|
||||
template String checkAndGetLiteralArgument(const ASTPtr &, const String &);
|
||||
template UInt64 checkAndGetLiteralArgument(const ASTPtr &, const String &);
|
||||
template String checkAndGetLiteralArgument(const ASTLiteral &, const String &);
|
||||
|
||||
}
|
21
src/Storages/checkAndGetLiteralArgument.h
Normal file
21
src/Storages/checkAndGetLiteralArgument.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T checkAndGetLiteralArgument(const ASTPtr & arg, const String & arg_name);
|
||||
|
||||
template <typename T>
|
||||
T checkAndGetLiteralArgument(const ASTLiteral & arg, const String & arg_name);
|
||||
|
||||
|
||||
|
||||
}
|
@ -2,11 +2,9 @@
|
||||
|
||||
#if USE_HIVE
|
||||
#include <memory>
|
||||
#include <type_traits>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ParserPartition.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
@ -14,6 +12,7 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Storages/Hive/HiveSettings.h>
|
||||
#include <Storages/Hive/StorageHive.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Common/logger_useful.h>
|
||||
@ -44,11 +43,11 @@ namespace DB
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_);
|
||||
|
||||
hive_metastore_url = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
hive_database = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
hive_table = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
table_structure = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
partition_by_def = args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
hive_metastore_url = checkAndGetLiteralArgument<String>(args[0], "hive_url");
|
||||
hive_database = checkAndGetLiteralArgument<String>(args[0], "hive_database");
|
||||
hive_table = checkAndGetLiteralArgument<String>(args[0], "hive_table");
|
||||
table_structure = checkAndGetLiteralArgument<String>(args[0], "structure");
|
||||
partition_by_def = checkAndGetLiteralArgument<String>(args[0], "partition_by_keys");
|
||||
|
||||
actual_columns = parseColumnsListFromString(table_structure, context_);
|
||||
}
|
||||
|
@ -8,11 +8,10 @@
|
||||
|
||||
#include <Storages/StorageFile.h>
|
||||
#include <Storages/Distributed/DirectoryMonitor.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <Processors/ISource.h>
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
namespace DB
|
||||
@ -25,10 +24,9 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, ContextPtr context)
|
||||
void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &)
|
||||
{
|
||||
auto ast = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
filename = ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
filename = checkAndGetLiteralArgument<String>(arg, "source");
|
||||
}
|
||||
|
||||
String ITableFunctionFileLike::getFormatFromFirstArgument()
|
||||
@ -49,13 +47,13 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
|
||||
if (args.empty())
|
||||
throw Exception("Table function '" + getName() + "' requires at least 1 argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
parseFirstArguments(args[0], context);
|
||||
|
||||
for (size_t i = 1; i < args.size(); ++i)
|
||||
args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context);
|
||||
|
||||
if (args.size() > 1)
|
||||
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
format = checkAndGetLiteralArgument<String>(args[1], "format");
|
||||
|
||||
if (format == "auto")
|
||||
format = getFormatFromFirstArgument();
|
||||
@ -67,7 +65,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
|
||||
throw Exception("Table function '" + getName() + "' requires 1, 2, 3 or 4 arguments: filename, format (default auto), structure (default auto) and compression method (default auto)",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
structure = checkAndGetLiteralArgument<String>(args[2], "structure");
|
||||
|
||||
if (structure.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -75,7 +73,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
|
||||
ast_function->formatForErrorMessage());
|
||||
|
||||
if (args.size() == 4)
|
||||
compression_method = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
compression_method = checkAndGetLiteralArgument<String>(args[3], "compression_method");
|
||||
}
|
||||
|
||||
StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
protected:
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
virtual void parseFirstArguments(const ASTPtr & arg, ContextPtr context);
|
||||
virtual void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context);
|
||||
virtual String getFormatFromFirstArgument();
|
||||
|
||||
String filename;
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
@ -35,7 +36,7 @@ void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, Contex
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
dictionary_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
dictionary_name = checkAndGetLiteralArgument<String>(args[0], "dictionary_name");
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const
|
||||
|
@ -4,8 +4,8 @@
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Storages/StorageExecutable.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -43,16 +43,16 @@ void TableFunctionExecutable::parseArguments(const ASTPtr & ast_function, Contex
|
||||
for (size_t i = 0; i <= 2; ++i)
|
||||
args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context);
|
||||
|
||||
auto scipt_name_with_arguments_value = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto script_name_with_arguments_value = checkAndGetLiteralArgument<String>(args[0], "script_name_with_arguments_value");
|
||||
|
||||
std::vector<String> script_name_with_arguments;
|
||||
boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c){ return c == ' '; });
|
||||
boost::split(script_name_with_arguments, script_name_with_arguments_value, [](char c){ return c == ' '; });
|
||||
|
||||
script_name = script_name_with_arguments[0];
|
||||
script_name_with_arguments.erase(script_name_with_arguments.begin());
|
||||
arguments = std::move(script_name_with_arguments);
|
||||
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
format = checkAndGetLiteralArgument<String>(args[1], "format");
|
||||
structure = checkAndGetLiteralArgument<String>(args[2], "structure");
|
||||
|
||||
for (size_t i = 3; i < args.size(); ++i)
|
||||
{
|
||||
|
@ -21,7 +21,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, ContextPtr context)
|
||||
void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr & context)
|
||||
{
|
||||
if (context->getApplicationType() != Context::ApplicationType::LOCAL)
|
||||
{
|
||||
@ -29,36 +29,27 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, ContextPtr conte
|
||||
return;
|
||||
}
|
||||
|
||||
if (auto opt_name = tryGetIdentifierName(arg))
|
||||
const auto * literal = arg->as<ASTLiteral>();
|
||||
auto type = literal->value.getType();
|
||||
if (type == Field::Types::String)
|
||||
{
|
||||
if (*opt_name == "stdin")
|
||||
filename = literal->value.safeGet<String>();
|
||||
if (filename == "stdin" || filename == "-")
|
||||
fd = STDIN_FILENO;
|
||||
else if (*opt_name == "stdout")
|
||||
else if (filename == "stdout")
|
||||
fd = STDOUT_FILENO;
|
||||
else if (*opt_name == "stderr")
|
||||
else if (filename == "stderr")
|
||||
fd = STDERR_FILENO;
|
||||
else
|
||||
filename = *opt_name;
|
||||
}
|
||||
else if (const auto * literal = arg->as<ASTLiteral>())
|
||||
else if (type == Field::Types::Int64 || type == Field::Types::UInt64)
|
||||
{
|
||||
auto type = literal->value.getType();
|
||||
if (type == Field::Types::Int64 || type == Field::Types::UInt64)
|
||||
{
|
||||
fd = (type == Field::Types::Int64) ? static_cast<int>(literal->value.get<Int64>()) : static_cast<int>(literal->value.get<UInt64>());
|
||||
if (fd < 0)
|
||||
throw Exception("File descriptor must be non-negative", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
else if (type == Field::Types::String)
|
||||
{
|
||||
filename = literal->value.get<String>();
|
||||
if (filename == "-")
|
||||
fd = STDIN_FILENO;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"The first argument of table function '" + getName() + "' mush be path or file descriptor", ErrorCodes::BAD_ARGUMENTS);
|
||||
fd = (type == Field::Types::Int64) ? literal->value.get<Int64>() : literal->value.get<UInt64>();
|
||||
if (fd < 0)
|
||||
throw Exception("File descriptor must be non-negative", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"The first argument of table function '" + getName() + "' mush be path or file descriptor", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
String TableFunctionFile::getFormatFromFirstArgument()
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
|
||||
protected:
|
||||
int fd = -1;
|
||||
void parseFirstArguments(const ASTPtr & arg, ContextPtr context) override;
|
||||
void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) override;
|
||||
String getFormatFromFirstArgument() override;
|
||||
|
||||
private:
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFormat.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
@ -43,8 +44,8 @@ void TableFunctionFormat::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
format = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
data = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
format = checkAndGetLiteralArgument<String>(args[0], "format");
|
||||
data = checkAndGetLiteralArgument<String>(args[1], "data");
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr context) const
|
||||
|
@ -3,6 +3,7 @@
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/StorageGenerateRandom.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -58,20 +59,20 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co
|
||||
}
|
||||
|
||||
/// Parsing first argument as table structure and creating a sample block
|
||||
structure = args[0]->as<const ASTLiteral &>().value.safeGet<String>();
|
||||
structure = checkAndGetLiteralArgument<String>(args[0], "structure");
|
||||
|
||||
if (args.size() >= 2)
|
||||
{
|
||||
const Field & value = args[1]->as<const ASTLiteral &>().value;
|
||||
if (!value.isNull())
|
||||
random_seed = value.safeGet<UInt64>();
|
||||
const auto & literal = args[1]->as<const ASTLiteral &>();
|
||||
if (!literal.value.isNull())
|
||||
random_seed = checkAndGetLiteralArgument<UInt64>(literal, "random_seed");
|
||||
}
|
||||
|
||||
if (args.size() >= 3)
|
||||
max_string_length = args[2]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||
max_string_length = checkAndGetLiteralArgument<UInt64>(args[2], "max_string_length");
|
||||
|
||||
if (args.size() == 4)
|
||||
max_array_length = args[3]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||
max_array_length = checkAndGetLiteralArgument<UInt64>(args[3], "max_string_length");
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context) const
|
||||
|
@ -5,8 +5,8 @@
|
||||
#include <Storages/HDFS/StorageHDFSCluster.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <QueryPipeline/RemoteQueryExecutor.h>
|
||||
#include <Storages/HDFS/StorageHDFS.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
@ -15,8 +15,6 @@
|
||||
#include <TableFunctions/TableFunctionHDFSCluster.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
@ -61,7 +59,7 @@ void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, Conte
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
/// This argument is always the first
|
||||
cluster_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
cluster_name = checkAndGetLiteralArgument<String>(args[0], "cluster_name");
|
||||
|
||||
if (!context->tryGetCluster(cluster_name))
|
||||
throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name);
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionInput.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
@ -6,6 +5,7 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Storages/StorageInput.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include "registerTableFunctions.h"
|
||||
@ -40,7 +40,7 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
throw Exception("Table function '" + getName() + "' requires exactly 1 argument: structure",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as<ASTLiteral &>().value.safeGet<String>();
|
||||
structure = checkAndGetLiteralArgument<String>(evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context), "structure");
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr context) const
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
#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>
|
||||
@ -58,10 +59,10 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
|
||||
if (!is_regexp)
|
||||
args[0] = database_ast;
|
||||
source_database_name_or_regexp = database_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
source_database_name_or_regexp = checkAndGetLiteralArgument<String>(database_ast, "database_name");
|
||||
|
||||
args[1] = evaluateConstantExpressionAsLiteral(args[1], context);
|
||||
source_table_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
source_table_regexp = checkAndGetLiteralArgument<String>(args[1], "table_name_regexp");
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,9 +1,9 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Storages/StorageNull.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionNull.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -30,7 +30,7 @@ void TableFunctionNull::parseArguments(const ASTPtr & ast_function, ContextPtr c
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments.empty())
|
||||
structure = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context)->as<ASTLiteral>()->value.safeGet<String>();
|
||||
structure = checkAndGetLiteralArgument<String>(evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context), "structure");
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionNull::getActualTableStructure(ContextPtr context) const
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <Storages/getStructureOfRemoteTable.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -13,7 +15,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <base/range.h>
|
||||
@ -79,7 +80,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
else
|
||||
{
|
||||
auto database_literal = evaluateConstantExpressionOrIdentifierAsLiteral(arg_value, context);
|
||||
configuration.database = database_literal->as<ASTLiteral>()->value.safeGet<String>();
|
||||
configuration.database = checkAndGetLiteralArgument<String>(database_literal, "database");
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -113,7 +114,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
if (is_cluster_function)
|
||||
{
|
||||
args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context);
|
||||
cluster_name = args[arg_num]->as<ASTLiteral &>().value.safeGet<const String &>();
|
||||
cluster_name = checkAndGetLiteralArgument<String>(args[arg_num], "cluster_name");
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -134,7 +135,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
else
|
||||
{
|
||||
args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context);
|
||||
configuration.database = args[arg_num]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.database = checkAndGetLiteralArgument<String>(args[arg_num], "database");
|
||||
|
||||
++arg_num;
|
||||
|
||||
@ -149,7 +150,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
{
|
||||
std::swap(qualified_name.database, qualified_name.table);
|
||||
args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context);
|
||||
qualified_name.table = args[arg_num]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
qualified_name.table = checkAndGetLiteralArgument<String>(args[arg_num], "table");
|
||||
++arg_num;
|
||||
}
|
||||
}
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <TableFunctions/TableFunctionS3.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include "registerTableFunctions.h"
|
||||
@ -56,7 +57,7 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar
|
||||
/// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not.
|
||||
if (args.size() == 4)
|
||||
{
|
||||
auto second_arg = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/access_key_id");
|
||||
if (FormatFactory::instance().getAllFormats().contains(second_arg))
|
||||
args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}};
|
||||
|
||||
@ -68,7 +69,8 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar
|
||||
/// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not.
|
||||
else if (args.size() == 3)
|
||||
{
|
||||
auto second_arg = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/access_key_id");
|
||||
if (FormatFactory::instance().getAllFormats().contains(second_arg))
|
||||
args_to_idx = {{"format", 1}, {"structure", 2}};
|
||||
else
|
||||
@ -80,22 +82,22 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar
|
||||
}
|
||||
|
||||
/// This argument is always the first
|
||||
s3_configuration.url = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
s3_configuration.url = checkAndGetLiteralArgument<String>(args[0], "url");
|
||||
|
||||
if (args_to_idx.contains("format"))
|
||||
s3_configuration.format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
s3_configuration.format = checkAndGetLiteralArgument<String>(args[args_to_idx["format"]], "format");
|
||||
|
||||
if (args_to_idx.contains("structure"))
|
||||
s3_configuration.structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
s3_configuration.structure = checkAndGetLiteralArgument<String>(args[args_to_idx["structure"]], "structure");
|
||||
|
||||
if (args_to_idx.contains("compression_method"))
|
||||
s3_configuration.compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
s3_configuration.compression_method = checkAndGetLiteralArgument<String>(args[args_to_idx["compression_method"]], "compression_method");
|
||||
|
||||
if (args_to_idx.contains("access_key_id"))
|
||||
s3_configuration.auth_settings.access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
s3_configuration.auth_settings.access_key_id = checkAndGetLiteralArgument<String>(args[args_to_idx["access_key_id"]], "access_key_id");
|
||||
|
||||
if (args_to_idx.contains("secret_access_key"))
|
||||
s3_configuration.auth_settings.secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
s3_configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument<String>(args[args_to_idx["secret_access_key"]], "secret_access_key");
|
||||
}
|
||||
|
||||
if (s3_configuration.format == "auto")
|
||||
|
@ -3,11 +3,11 @@
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Storages/StorageS3Cluster.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <QueryPipeline/RemoteQueryExecutor.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
@ -17,7 +17,6 @@
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
@ -65,7 +64,7 @@ void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, Context
|
||||
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
/// This arguments are always the first
|
||||
configuration.cluster_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
configuration.cluster_name = checkAndGetLiteralArgument<String>(args[0], "cluster_name");
|
||||
|
||||
if (!context->tryGetCluster(configuration.cluster_name))
|
||||
throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", configuration.cluster_name);
|
||||
|
@ -10,14 +10,14 @@
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -73,8 +73,8 @@ void TableFunctionSQLite::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
database_path = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
remote_table_name = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
database_path = checkAndGetLiteralArgument<String>(args[0], "database_path");
|
||||
remote_table_name = checkAndGetLiteralArgument<String>(args[0], "table_name");
|
||||
|
||||
sqlite_db = openSQLiteDB(database_path, context);
|
||||
}
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <TableFunctions/TableFunctionZeros.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Storages/System/StorageSystemZeros.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -55,7 +55,7 @@ void registerTableFunctionZeros(TableFunctionFactory & factory)
|
||||
template <bool multithreaded>
|
||||
UInt64 TableFunctionZeros<multithreaded>::evaluateArgument(ContextPtr context, ASTPtr & argument) const
|
||||
{
|
||||
return evaluateConstantExpressionOrIdentifierAsLiteral(argument, context)->as<ASTLiteral &>().value.safeGet<UInt64>();
|
||||
return checkAndGetLiteralArgument<UInt64>(evaluateConstantExpressionOrIdentifierAsLiteral(argument, context), "length");
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user