mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Review fixes
This commit is contained in:
parent
36d0f67b38
commit
e04e37916e
@ -1,18 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename ValueType>
|
||||
static inline ValueType safeGetFromASTLiteral(const ASTPtr & ast)
|
||||
{
|
||||
return ast->as<ASTLiteral>()->value.safeGet<ValueType>();
|
||||
}
|
||||
|
||||
static inline bool isFunctionCast(const ASTFunction * function)
|
||||
{
|
||||
if (function)
|
||||
|
@ -111,7 +111,7 @@ std::optional<ExternalDataSourceConfig> getExternalDataSourceConfiguration(const
|
||||
}
|
||||
|
||||
auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context);
|
||||
auto * arg_value_literal = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as<ASTLiteral>();
|
||||
auto * arg_value_literal = arg_value_ast->as<ASTLiteral>();
|
||||
if (arg_value_literal)
|
||||
{
|
||||
auto arg_value = arg_value_literal->value;
|
||||
@ -315,7 +315,7 @@ std::optional<URLBasedDataSourceConfig> getURLBasedDataSourceConfiguration(const
|
||||
|
||||
auto arg_name = function_args[0]->as<ASTIdentifier>()->name();
|
||||
auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context);
|
||||
auto arg_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as<ASTLiteral>()->value;
|
||||
auto arg_value = arg_value_ast->as<ASTLiteral>()->value;
|
||||
|
||||
if (arg_name == "url")
|
||||
configuration.url = arg_value.safeGet<String>();
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
@ -234,7 +233,7 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
for (const auto & [name, value] : storage_specific_args)
|
||||
{
|
||||
if (name == "description")
|
||||
cluster_description = safeGetFromASTLiteral<String>(value);
|
||||
cluster_description = value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unknown key-value argument {} for table engine URL", name);
|
||||
@ -280,7 +279,7 @@ void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
for (const auto & [name, value] : storage_specific_args)
|
||||
{
|
||||
if (name == "description")
|
||||
cluster_description = safeGetFromASTLiteral<String>(value);
|
||||
cluster_description = value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unknown key-value argument {} for table function URL", name);
|
||||
|
@ -12,7 +12,6 @@
|
||||
#include <Common/parseAddress.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/Transforms/MongoDBSource.h>
|
||||
|
||||
@ -114,9 +113,9 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C
|
||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
||||
{
|
||||
if (arg_name == "collection")
|
||||
configuration.collection = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.collection = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else if (arg_name == "options")
|
||||
configuration.options = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.options = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key-value argument."
|
||||
|
@ -16,7 +16,6 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <mysqlxx/Transaction.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
@ -251,9 +250,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 = safeGetFromASTLiteral<bool>(arg_value);
|
||||
configuration.replace_query = arg_value->as<ASTLiteral>()->value.safeGet<bool>();
|
||||
else if (arg_name == "on_duplicate_clause")
|
||||
configuration.on_duplicate_clause = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.on_duplicate_clause = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key-value argument."
|
||||
|
@ -38,7 +38,6 @@
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -399,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 = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.on_conflict = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key-value argument."
|
||||
|
@ -18,7 +18,6 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
@ -751,9 +750,9 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
|
||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
||||
{
|
||||
if (arg_name == "access_key_id")
|
||||
configuration.access_key_id = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.access_key_id = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else if (arg_name == "secret_access_key")
|
||||
configuration.secret_access_key = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.secret_access_key = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
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].",
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -80,7 +79,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
else
|
||||
{
|
||||
auto database_literal = evaluateConstantExpressionOrIdentifierAsLiteral(arg_value, context);
|
||||
configuration.database = safeGetFromASTLiteral<String>(database_literal);
|
||||
configuration.database = database_literal->as<ASTLiteral>()->value.safeGet<String>();
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionS3.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
#include "registerTableFunctions.h"
|
||||
@ -49,9 +48,9 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con
|
||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
||||
{
|
||||
if (arg_name == "access_key_id")
|
||||
configuration.access_key_id = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.access_key_id = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else if (arg_name == "secret_access_key")
|
||||
configuration.secret_access_key = safeGetFromASTLiteral<String>(arg_value);
|
||||
configuration.secret_access_key = arg_value->as<ASTLiteral>()->value.safeGet<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Unknown key-value argument `{}` for StorageS3, expected: "
|
||||
|
@ -6,6 +6,6 @@ INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(1);
|
||||
INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(2);
|
||||
INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(3);
|
||||
INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(4);
|
||||
SELECT COUNT(*) FROM remote(remote1, database=currentDatabase());
|
||||
SELECT count(*) FROM remote(remote2, database=merge(currentDatabase(), '^remote_test'));
|
||||
SELECT count() FROM remote(remote1, database=currentDatabase());
|
||||
SELECT count() FROM remote(remote2, database=merge(currentDatabase(), '^remote_test'));
|
||||
DROP TABLE remote_test;
|
||||
|
Loading…
Reference in New Issue
Block a user