This commit is contained in:
Danila Kutenin 2019-01-15 16:19:15 +03:00
commit 43965c1703
84 changed files with 1173 additions and 750 deletions

3
.gitignore vendored
View File

@ -251,3 +251,6 @@ website/package-lock.json
# cquery cache
/.cquery-cache
# ccls cache
/.ccls-cache

3
.gitmodules vendored
View File

@ -61,3 +61,6 @@
[submodule "contrib/libgsasl"]
path = contrib/libgsasl
url = https://github.com/ClickHouse-Extras/libgsasl.git
[submodule "contrib/cppkafka"]
path = contrib/cppkafka
url = https://github.com/mfontanini/cppkafka.git

View File

@ -20,11 +20,13 @@ if (NOT USE_INTERNAL_RDKAFKA_LIBRARY)
if (USE_STATIC_LIBRARIES AND NOT OS_FREEBSD)
find_library (SASL2_LIBRARY sasl2)
endif ()
set (CPPKAFKA_LIBRARY cppkafka) # TODO: try to use unbundled version.
endif ()
if (RDKAFKA_LIB AND RDKAFKA_INCLUDE_DIR)
set (USE_RDKAFKA 1)
set (RDKAFKA_LIBRARY ${RDKAFKA_LIB} ${OPENSSL_LIBRARIES})
set (CPPKAFKA_LIBRARY cppkafka)
if (SASL2_LIBRARY)
list (APPEND RDKAFKA_LIBRARY ${SASL2_LIBRARY})
endif ()
@ -35,9 +37,10 @@ elseif (NOT MISSING_INTERNAL_RDKAFKA_LIBRARY AND NOT ARCH_ARM)
set (USE_INTERNAL_RDKAFKA_LIBRARY 1)
set (RDKAFKA_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src")
set (RDKAFKA_LIBRARY rdkafka)
set (CPPKAFKA_LIBRARY cppkafka)
set (USE_RDKAFKA 1)
endif ()
endif ()
message (STATUS "Using librdkafka=${USE_RDKAFKA}: ${RDKAFKA_INCLUDE_DIR} : ${RDKAFKA_LIBRARY}")
message (STATUS "Using librdkafka=${USE_RDKAFKA}: ${RDKAFKA_INCLUDE_DIR} : ${RDKAFKA_LIBRARY} ${CPPKAFKA_LIBRARY}")

View File

@ -125,6 +125,10 @@ if (USE_INTERNAL_RDKAFKA_LIBRARY)
target_include_directories(rdkafka BEFORE PRIVATE ${OPENSSL_INCLUDE_DIR})
endif ()
if (USE_RDKAFKA)
add_subdirectory (cppkafka-cmake)
endif()
if (ENABLE_ODBC AND USE_INTERNAL_ODBC_LIBRARY)
add_subdirectory (unixodbc-cmake)
endif ()

1
contrib/cppkafka vendored Submodule

@ -0,0 +1 @@
Subproject commit 520465510efef7704346cf8d140967c4abb057c1

View File

@ -0,0 +1,31 @@
set(CPPKAFKA_DIR ${CMAKE_SOURCE_DIR}/contrib/cppkafka)
set(SRCS
${CPPKAFKA_DIR}/src/configuration.cpp
${CPPKAFKA_DIR}/src/topic_configuration.cpp
${CPPKAFKA_DIR}/src/configuration_option.cpp
${CPPKAFKA_DIR}/src/exceptions.cpp
${CPPKAFKA_DIR}/src/topic.cpp
${CPPKAFKA_DIR}/src/buffer.cpp
${CPPKAFKA_DIR}/src/queue.cpp
${CPPKAFKA_DIR}/src/message.cpp
${CPPKAFKA_DIR}/src/message_timestamp.cpp
${CPPKAFKA_DIR}/src/message_internal.cpp
${CPPKAFKA_DIR}/src/topic_partition.cpp
${CPPKAFKA_DIR}/src/topic_partition_list.cpp
${CPPKAFKA_DIR}/src/metadata.cpp
${CPPKAFKA_DIR}/src/group_information.cpp
${CPPKAFKA_DIR}/src/error.cpp
${CPPKAFKA_DIR}/src/event.cpp
${CPPKAFKA_DIR}/src/kafka_handle_base.cpp
${CPPKAFKA_DIR}/src/producer.cpp
${CPPKAFKA_DIR}/src/consumer.cpp
)
add_library(cppkafka ${LINK_MODE} ${SRCS})
target_link_libraries(cppkafka PRIVATE ${RDKAFKA_LIBRARY})
target_include_directories(cppkafka PRIVATE ${CPPKAFKA_DIR}/include/cppkafka)
target_include_directories(cppkafka PRIVATE ${Boost_INCLUDE_DIRS})
target_include_directories(cppkafka SYSTEM PUBLIC ${CPPKAFKA_DIR}/include)

2
contrib/librdkafka vendored

@ -1 +1 @@
Subproject commit 7478b5ef16aadd6543fe38bc6a2deb895c70da98
Subproject commit 363dcad5a23dc29381cc626620e68ae418b3af19

View File

@ -1,60 +1,63 @@
set(RDKAFKA_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/librdkafka/src)
set(SRCS
${RDKAFKA_SOURCE_DIR}/crc32c.c
${RDKAFKA_SOURCE_DIR}/rdaddr.c
${RDKAFKA_SOURCE_DIR}/rdavl.c
${RDKAFKA_SOURCE_DIR}/rdbuf.c
${RDKAFKA_SOURCE_DIR}/rdcrc32.c
${RDKAFKA_SOURCE_DIR}/rdkafka.c
${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_broker.c
${RDKAFKA_SOURCE_DIR}/rdkafka_buf.c
${RDKAFKA_SOURCE_DIR}/rdkafka_cgrp.c
${RDKAFKA_SOURCE_DIR}/rdkafka_conf.c
${RDKAFKA_SOURCE_DIR}/rdkafka_event.c
${RDKAFKA_SOURCE_DIR}/rdkafka_feature.c
${RDKAFKA_SOURCE_DIR}/rdkafka_lz4.c
${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c
${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_writer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_offset.c
${RDKAFKA_SOURCE_DIR}/rdkafka_op.c
${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c
${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c
${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c
${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_request.c
${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c
${RDKAFKA_SOURCE_DIR}/rdkafka_subscription.c
${RDKAFKA_SOURCE_DIR}/rdkafka_timer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_topic.c
${RDKAFKA_SOURCE_DIR}/rdkafka_transport.c
${RDKAFKA_SOURCE_DIR}/rdkafka_interceptor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_header.c
${RDKAFKA_SOURCE_DIR}/rdlist.c
${RDKAFKA_SOURCE_DIR}/rdlog.c
${RDKAFKA_SOURCE_DIR}/rdmurmur2.c
${RDKAFKA_SOURCE_DIR}/rdports.c
${RDKAFKA_SOURCE_DIR}/rdrand.c
${RDKAFKA_SOURCE_DIR}/rdregex.c
${RDKAFKA_SOURCE_DIR}/rdstring.c
${RDKAFKA_SOURCE_DIR}/rdunittest.c
${RDKAFKA_SOURCE_DIR}/rdvarint.c
${RDKAFKA_SOURCE_DIR}/snappy.c
${RDKAFKA_SOURCE_DIR}/tinycthread.c
${RDKAFKA_SOURCE_DIR}/xxhash.c
${RDKAFKA_SOURCE_DIR}/lz4.c
${RDKAFKA_SOURCE_DIR}/lz4frame.c
${RDKAFKA_SOURCE_DIR}/lz4hc.c
${RDKAFKA_SOURCE_DIR}/rdgz.c
${RDKAFKA_SOURCE_DIR}/crc32c.c
${RDKAFKA_SOURCE_DIR}/rdaddr.c
${RDKAFKA_SOURCE_DIR}/rdavl.c
${RDKAFKA_SOURCE_DIR}/rdbuf.c
${RDKAFKA_SOURCE_DIR}/rdcrc32.c
${RDKAFKA_SOURCE_DIR}/rdkafka.c
${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_background.c
${RDKAFKA_SOURCE_DIR}/rdkafka_broker.c
${RDKAFKA_SOURCE_DIR}/rdkafka_buf.c
${RDKAFKA_SOURCE_DIR}/rdkafka_cgrp.c
${RDKAFKA_SOURCE_DIR}/rdkafka_conf.c
${RDKAFKA_SOURCE_DIR}/rdkafka_event.c
${RDKAFKA_SOURCE_DIR}/rdkafka_feature.c
${RDKAFKA_SOURCE_DIR}/rdkafka_idempotence.c
${RDKAFKA_SOURCE_DIR}/rdkafka_lz4.c
${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c
${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_writer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_offset.c
${RDKAFKA_SOURCE_DIR}/rdkafka_op.c
${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c
${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c
${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c
${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_request.c
${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c
${RDKAFKA_SOURCE_DIR}/rdkafka_subscription.c
${RDKAFKA_SOURCE_DIR}/rdkafka_timer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_topic.c
${RDKAFKA_SOURCE_DIR}/rdkafka_transport.c
${RDKAFKA_SOURCE_DIR}/rdkafka_interceptor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_header.c
${RDKAFKA_SOURCE_DIR}/rdlist.c
${RDKAFKA_SOURCE_DIR}/rdlog.c
${RDKAFKA_SOURCE_DIR}/rdmurmur2.c
${RDKAFKA_SOURCE_DIR}/rdports.c
${RDKAFKA_SOURCE_DIR}/rdrand.c
${RDKAFKA_SOURCE_DIR}/rdregex.c
${RDKAFKA_SOURCE_DIR}/rdstring.c
${RDKAFKA_SOURCE_DIR}/rdunittest.c
${RDKAFKA_SOURCE_DIR}/rdvarint.c
${RDKAFKA_SOURCE_DIR}/snappy.c
${RDKAFKA_SOURCE_DIR}/tinycthread.c
${RDKAFKA_SOURCE_DIR}/tinycthread_extra.c
${RDKAFKA_SOURCE_DIR}/xxhash.c
${RDKAFKA_SOURCE_DIR}/lz4.c
${RDKAFKA_SOURCE_DIR}/lz4frame.c
${RDKAFKA_SOURCE_DIR}/lz4hc.c
${RDKAFKA_SOURCE_DIR}/rdgz.c
)
add_library(rdkafka ${LINK_MODE} ${SRCS})
target_include_directories(rdkafka PRIVATE include)
target_include_directories(rdkafka SYSTEM PUBLIC include)
target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR})
target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY})

View File

@ -1,4 +1,4 @@
// Automatically generated by ./configure
// Automatically generated by ./configure
#ifndef _CONFIG_H_
#define _CONFIG_H_
#define ARCH "x86_64"
@ -71,4 +71,8 @@
#define HAVE_PTHREAD_SETNAME_GNU 1
// python
//#define HAVE_PYTHON 1
// C11 threads
#if (__STDC_VERSION__ >= 201112L) && !defined(__STDC_NO_THREADS__)
# define WITH_C11THREADS 1
#endif
#endif /* _CONFIG_H_ */

View File

@ -0,0 +1,5 @@
#if __has_include(<rdkafka.h>) // maybe bundled
# include_next <rdkafka.h> // Y_IGNORE
#else // system
# include_next <librdkafka/rdkafka.h>
#endif

View File

@ -287,6 +287,7 @@ endif ()
if (USE_RDKAFKA)
target_link_libraries (dbms PRIVATE ${RDKAFKA_LIBRARY})
target_link_libraries (dbms PRIVATE ${CPPKAFKA_LIBRARY})
if (NOT USE_INTERNAL_RDKAFKA_LIBRARY)
target_include_directories (dbms SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR})
endif ()

View File

@ -5,6 +5,7 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/hex.h>
#include <sstream>
namespace DB
@ -20,14 +21,16 @@ extern const int CORRUPTED_DATA;
CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs)
: codecs(codecs)
{
std::ostringstream ss;
for (size_t idx = 0; idx < codecs.size(); idx++)
{
if (idx != 0)
codec_desc = codec_desc + ',';
ss << ',' << ' ';
const auto codec = codecs[idx];
codec_desc = codec_desc + codec->getCodecDesc();
ss << codec->getCodecDesc();
}
codec_desc = ss.str();
}
UInt8 CompressionCodecMultiple::getMethodByte() const

View File

@ -317,9 +317,9 @@ static DataTypePtr create(const ASTPtr & arguments)
params_row[i] = lit->value;
}
}
else if (const ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(arguments->children[0].get()))
else if (auto opt_name = getIdentifierName(arguments->children[0]))
{
function_name = identifier->name;
function_name = *opt_name;
}
else if (typeid_cast<ASTLiteral *>(arguments->children[0].get()))
{

View File

@ -7,6 +7,7 @@
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabaseMemory.h>
#include <Databases/DatabasesCommon.h>
#include <Common/typeid_cast.h>
#include <Common/escapeForFileName.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Stopwatch.h>

View File

@ -1,5 +1,6 @@
#include <sstream>
#include <Common/typeid_cast.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>

View File

@ -103,7 +103,7 @@ private:
void visit(const ASTIdentifier & identifier, ASTPtr & ast) const
{
if (ast->children.empty())
ast = createDatabaseAndTableNode(database_name, identifier.name);
ast = createTableIdentifier(database_name, identifier.name);
}
void visit(ASTSubquery & subquery, ASTPtr &) const

View File

@ -7,7 +7,6 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/IStorage.h>
@ -118,8 +117,7 @@ NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpressio
}
else if (table_expression.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
DatabaseAndTableWithAlias database_table(table_expression.database_and_table_name);
const auto & table = context.getTable(database_table.database, database_table.table);
names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList();
}

View File

@ -58,7 +58,7 @@ private:
NameToNameMap & array_join_alias_to_name = data.array_join_alias_to_name;
NameToNameMap & array_join_result_to_source = data.array_join_result_to_source;
if (!node.general())
if (!getColumnIdentifierName(node))
return;
auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1

View File

@ -31,7 +31,7 @@ bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public)
void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public)
{
if (!node.general())
if (!getColumnIdentifierName(node))
return;
required_names.insert(node.name);

View File

@ -39,9 +39,7 @@ struct ColumnNamesContext
std::optional<String> name() const
{
if (expr)
if (auto * node = expr->database_and_table_name.get())
if (auto * identifier = typeid_cast<const ASTIdentifier *>(node))
return identifier->name;
return getIdentifierName(expr->database_and_table_name);
return {};
}

View File

@ -54,8 +54,6 @@ size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifie
{
size_t num_qualifiers_to_strip = 0;
auto get_identifier_name = [](const ASTPtr & ast) { return static_cast<const ASTIdentifier &>(*ast).name; };
/// It is compound identifier
if (!identifier.children.empty())
{
@ -64,16 +62,16 @@ size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifie
/// database.table.column
if (num_components >= 3
&& !names.database.empty()
&& get_identifier_name(identifier.children[0]) == names.database
&& get_identifier_name(identifier.children[1]) == names.table)
&& *getIdentifierName(identifier.children[0]) == names.database
&& *getIdentifierName(identifier.children[1]) == names.table)
{
num_qualifiers_to_strip = 2;
}
/// table.column or alias.column. If num_components > 2, it is like table.nested.column.
if (num_components >= 2
&& ((!names.table.empty() && get_identifier_name(identifier.children[0]) == names.table)
|| (!names.alias.empty() && get_identifier_name(identifier.children[0]) == names.alias)))
&& ((!names.table.empty() && *getIdentifierName(identifier.children[0]) == names.table)
|| (!names.alias.empty() && *getIdentifierName(identifier.children[0]) == names.alias)))
{
num_qualifiers_to_strip = 1;
}
@ -94,26 +92,24 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
if (identifier.children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
const ASTIdentifier * db_identifier = typeid_cast<const ASTIdentifier *>(identifier.children[0].get());
const ASTIdentifier * table_identifier = typeid_cast<const ASTIdentifier *>(identifier.children[1].get());
if (!db_identifier || !table_identifier)
throw Exception("Logical error: identifiers expected", ErrorCodes::LOGICAL_ERROR);
database = db_identifier->name;
table = table_identifier->name;
getIdentifierName(identifier.children[0], database);
getIdentifierName(identifier.children[1], table);
}
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
{
const auto * identifier = typeid_cast<const ASTIdentifier *>(node.get());
if (!identifier)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
*this = DatabaseAndTableWithAlias(*identifier, current_database);
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database)
{
if (table_expression.database_and_table_name)
{
const auto * identifier = typeid_cast<const ASTIdentifier *>(table_expression.database_and_table_name.get());
if (!identifier)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
*this = DatabaseAndTableWithAlias(*identifier, current_database);
}
*this = DatabaseAndTableWithAlias(table_expression.database_and_table_name, current_database);
else if (table_expression.table_function)
alias = table_expression.table_function->tryGetAlias();
else if (table_expression.subquery)
@ -207,14 +203,10 @@ std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuer
return {};
ASTPtr database_and_table_name = table_expression->database_and_table_name;
if (!database_and_table_name)
if (!database_and_table_name || !isIdentifier(database_and_table_name))
return {};
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(database_and_table_name.get());
if (!identifier)
return {};
return *identifier;
return DatabaseAndTableWithAlias(database_and_table_name);
}
ASTPtr getTableFunctionOrSubquery(const ASTSelectQuery & select, size_t table_number)

View File

@ -23,6 +23,8 @@ struct DatabaseAndTableWithAlias
String table;
String alias;
DatabaseAndTableWithAlias() = default;
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database);

View File

@ -78,9 +78,6 @@ namespace ErrorCodes
extern const int EXPECTED_ALL_OR_ANY;
}
/// From SyntaxAnalyzer.cpp
extern void removeDuplicateColumns(NamesAndTypesList & columns);
ExpressionAnalyzer::ExpressionAnalyzer(
const ASTPtr & query_,
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
@ -551,8 +548,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
/// TODO This syntax does not support specifying a database name.
if (table_to_join.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
DatabaseAndTableWithAlias database_table(table_to_join.database_and_table_name);
StoragePtr table = context.tryGetTable(database_table.database, database_table.table);
if (table)

View File

@ -33,9 +33,9 @@ public:
private:
static std::vector<ASTPtr *> visit(const ASTIdentifier & node, ASTPtr &, Data & data)
{
if (node.special())
if (StoragePtr external_storage = data.context.tryGetExternalTable(node.name))
data.external_tables[node.name] = external_storage;
if (auto opt_name = getTableIdentifierName(node))
if (StoragePtr external_storage = data.context.tryGetExternalTable(*opt_name))
data.external_tables[*opt_name] = external_storage;
return {};
}
};

View File

@ -112,7 +112,7 @@ public:
* instead of doing a subquery, you just need to read it.
*/
auto database_and_table_name = createDatabaseAndTableNode("", external_table_name);
auto database_and_table_name = createTableIdentifier("", external_table_name);
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
{

View File

@ -2,10 +2,10 @@
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Storages/StorageDistributed.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
@ -82,12 +82,7 @@ void forEachTable(IAST * node, F && f)
StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & context)
{
const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(database_and_table.get());
if (!id)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
DatabaseAndTableWithAlias db_and_table(*id);
DatabaseAndTableWithAlias db_and_table(database_and_table);
return context.tryGetTable(db_and_table.database, db_and_table.table);
}
@ -173,8 +168,7 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
std::string table;
std::tie(database, table) = getRemoteDatabaseAndTableName(*storage);
/// TODO: find a way to avoid AST node replacing
database_and_table = createDatabaseAndTableNode(database, table);
database_and_table = createTableIdentifier(database, table);
}
else
throw Exception("InJoinSubqueriesPreprocessor: unexpected value of 'distributed_product_mode' setting", ErrorCodes::LOGICAL_ERROR);

View File

@ -95,13 +95,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
auto database_ptr = identifier->children[0];
auto table_ptr = identifier->children[1];
if (database_ptr)
database_name = typeid_cast<ASTIdentifier &>(*database_ptr).name;
if (table_ptr)
table_name = typeid_cast<ASTIdentifier &>(*table_ptr).name;
getIdentifierName(database_ptr, database_name);
getIdentifierName(table_ptr, table_name);
}
else
table_name = typeid_cast<ASTIdentifier &>(*identifier).name;
getIdentifierName(identifier, table_name);
table = context.getTable(database_name, table_name);
}

View File

@ -83,8 +83,8 @@ static void appendTableNameAndAlias(std::vector<String> & hidden, const ASTPtr &
if (!alias.empty())
hidden.push_back(alias);
if (auto * identifier = typeid_cast<const ASTIdentifier *>(table_expression->children[0].get()))
hidden.push_back(identifier->name);
if (auto opt_name = getIdentifierName(table_expression->children[0]))
hidden.push_back(*opt_name);
else if (alias.empty())
throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -319,9 +319,12 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
TranslateQualifiedNamesVisitor::Data qn_visitor_data{{}, tables};
TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast);
QueryAliasesVisitor::Data query_aliases_data{aliases};
QueryAliasesVisitor(query_aliases_data).visit(ast);
QueryNormalizer(ast, aliases, settings, {}, {}).perform();
QueryNormalizer::Data normalizer_data(aliases, settings);
QueryNormalizer(normalizer_data).visit(ast);
for (const auto & projection_column : select_query->select_expression_list->children)
{

View File

@ -20,39 +20,253 @@ namespace ErrorCodes
extern const int CYCLIC_ALIASES;
}
QueryNormalizer::QueryNormalizer(ASTPtr & query, const QueryNormalizer::Aliases & aliases,
ExtractedSettings && settings_, const Names & all_column_names,
const TableNamesAndColumnNames & table_names_and_column_names)
: query(query), aliases(aliases), settings(settings_), all_column_names(all_column_names),
table_names_and_column_names(table_names_and_column_names)
class CheckASTDepth
{
}
void QueryNormalizer::perform()
{
SetOfASTs tmp_set;
MapOfASTs tmp_map;
performImpl(query, tmp_map, tmp_set, "", 0);
try
public:
CheckASTDepth(QueryNormalizer::Data & data_)
: data(data_)
{
query->checkSize(settings.max_expanded_ast_elements);
if (data.level > data.settings.max_ast_depth)
throw Exception("Normalized AST is too deep. Maximum: " + toString(data.settings.max_ast_depth), ErrorCodes::TOO_DEEP_AST);
++data.level;
}
catch (Exception & e)
~CheckASTDepth()
{
e.addMessage("(after expansion of aliases)");
throw;
--data.level;
}
private:
QueryNormalizer::Data & data;
};
class RestoreAliasOnExitScope
{
public:
RestoreAliasOnExitScope(String & alias_)
: alias(alias_)
, copy(alias_)
{}
~RestoreAliasOnExitScope()
{
alias = copy;
}
private:
String & alias;
const String copy;
};
void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data)
{
auto & aliases = data.aliases;
String & func_name = node.name;
ASTPtr & func_arguments = node.arguments;
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (functionIsInOrGlobalInOperator(func_name))
{
auto & ast = func_arguments->children.at(1);
if (auto opt_name = getIdentifierName(ast))
if (!aliases.count(*opt_name))
setIdentifierSpecial(ast);
}
/// Special cases for count function.
String func_name_lowercase = Poco::toLower(func_name);
if (startsWith(func_name_lowercase, "count"))
{
/// Select implementation of countDistinct based on settings.
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
if (endsWith(func_name, "Distinct") && func_name_lowercase == "countdistinct")
func_name = data.settings.count_distinct_implementation;
/// As special case, treat count(*) as count(), not as count(list of all columns).
if (func_name_lowercase == "count" && func_arguments->children.size() == 1
&& typeid_cast<const ASTAsterisk *>(func_arguments->children[0].get()))
{
func_arguments->children.clear();
}
}
}
/// finished_asts - already processed vertices (and by what they replaced)
/// current_asts - vertices in the current call stack of this method
/// current_alias - the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level)
void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
{
if (level > settings.max_ast_depth)
throw Exception("Normalized AST is too deep. Maximum: " + toString(settings.max_ast_depth), ErrorCodes::TOO_DEEP_AST);
auto & current_asts = data.current_asts;
String & current_alias = data.current_alias;
if (!getColumnIdentifierName(node))
return;
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
auto it_alias = data.aliases.find(node.name);
if (it_alias != data.aliases.end() && current_alias != node.name)
{
auto & alias_node = it_alias->second;
/// Let's replace it with the corresponding tree node.
if (current_asts.count(alias_node.get()))
throw Exception("Cyclic aliases", ErrorCodes::CYCLIC_ALIASES);
String my_alias = ast->tryGetAlias();
if (!my_alias.empty() && my_alias != alias_node->getAliasOrColumnName())
{
/// Avoid infinite recursion here
auto opt_name = getColumnIdentifierName(alias_node);
bool is_cycle = opt_name && *opt_name == node.name;
if (!is_cycle)
{
/// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a.
ast = alias_node->clone();
ast->setAlias(my_alias);
}
}
else
ast = alias_node;
}
}
/// Replace *, alias.*, database.table.* with a list of columns.
void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & data)
{
auto & tables_with_columns = data.tables_with_columns;
ASTs old_children;
if (data.processAsterisks())
{
bool has_asterisk = false;
for (const auto & child : node.children)
{
if (typeid_cast<const ASTAsterisk *>(child.get()) ||
typeid_cast<const ASTQualifiedAsterisk *>(child.get()))
{
has_asterisk = true;
break;
}
}
if (has_asterisk)
{
old_children.swap(node.children);
node.children.reserve(old_children.size());
}
}
for (const auto & child : old_children)
{
if (typeid_cast<const ASTAsterisk *>(child.get()))
{
for (const auto & pr : tables_with_columns)
for (const auto & column_name : pr.second)
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
}
else if (const auto * qualified_asterisk = typeid_cast<const ASTQualifiedAsterisk *>(child.get()))
{
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(qualified_asterisk->children[0].get());
size_t num_components = identifier->children.size();
for (const auto & [table_name, table_columns] : tables_with_columns)
{
if ((num_components == 2 /// database.table.*
&& !table_name.database.empty() /// This is normal (not a temporary) table.
&& static_cast<const ASTIdentifier &>(*identifier->children[0]).name == table_name.database
&& static_cast<const ASTIdentifier &>(*identifier->children[1]).name == table_name.table)
|| (num_components == 0 /// t.*
&& ((!table_name.table.empty() && identifier->name == table_name.table) /// table.*
|| (!table_name.alias.empty() && identifier->name == table_name.alias)))) /// alias.*
{
for (const auto & column_name : table_columns)
node.children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
break;
}
}
}
else
node.children.emplace_back(child);
}
}
/// mark table identifiers as 'not columns'
void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data &)
{
if (node.table_expression)
{
auto & expr = static_cast<ASTTableExpression &>(*node.table_expression);
setIdentifierSpecial(expr.database_and_table_name);
}
}
/// special visitChildren() for ASTSelectQuery
void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr & ast, Data & data)
{
for (auto & child : ast->children)
{
if (typeid_cast<const ASTSelectQuery *>(child.get()) ||
typeid_cast<const ASTTableExpression *>(child.get()))
continue;
visit(child, data);
}
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children,
/// but also in where_expression and having_expression.
if (select.prewhere_expression)
visit(select.prewhere_expression, data);
if (select.where_expression)
visit(select.where_expression, data);
if (select.having_expression)
visit(select.having_expression, data);
}
/// Don't go into subqueries.
/// Don't go into components of compound identifiers.
/// Don't go into select query. It processes children itself.
/// Do not go to the left argument of lambda expressions, so as not to replace the formal parameters
/// on aliases in expressions of the form 123 AS x, arrayMap(x -> 1, [2]).
void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data)
{
ASTFunction * func_node = typeid_cast<ASTFunction *>(node.get());
if (func_node && func_node->name == "lambda")
{
/// We skip the first argument. We also assume that the lambda function can not have parameters.
for (size_t i = 1, size = func_node->arguments->children.size(); i < size; ++i)
{
auto & child = func_node->arguments->children[i];
if (typeid_cast<const ASTSelectQuery *>(child.get()) ||
typeid_cast<const ASTTableExpression *>(child.get()))
continue;
visit(child, data);
}
}
else if (!typeid_cast<ASTIdentifier *>(node.get()) &&
!typeid_cast<ASTSelectQuery *>(node.get()))
{
for (auto & child : node->children)
{
if (typeid_cast<const ASTSelectQuery *>(child.get()) ||
typeid_cast<const ASTTableExpression *>(child.get()))
continue;
visit(child, data);
}
}
}
void QueryNormalizer::visit(ASTPtr & ast, Data & data)
{
CheckASTDepth scope1(data);
RestoreAliasOnExitScope scope2(data.current_alias);
auto & finished_asts = data.finished_asts;
auto & current_asts = data.current_asts;
if (finished_asts.count(ast))
{
@ -63,185 +277,46 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
ASTPtr initial_ast = ast;
current_asts.insert(initial_ast.get());
String my_alias = ast->tryGetAlias();
if (!my_alias.empty())
current_alias = my_alias;
/// rewrite rules that act when you go from top to bottom.
bool replaced = false;
ASTIdentifier * identifier_node = nullptr;
ASTFunction * func_node = nullptr;
if ((func_node = typeid_cast<ASTFunction *>(ast.get())))
{
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (functionIsInOrGlobalInOperator(func_node->name))
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(func_node->arguments->children.at(1).get()))
if (!aliases.count(right->name))
right->setSpecial();
/// Special cases for count function.
String func_name_lowercase = Poco::toLower(func_node->name);
if (startsWith(func_name_lowercase, "count"))
{
/// Select implementation of countDistinct based on settings.
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
if (endsWith(func_node->name, "Distinct") && func_name_lowercase == "countdistinct")
func_node->name = settings.count_distinct_implementation;
/// As special case, treat count(*) as count(), not as count(list of all columns).
if (func_name_lowercase == "count" && func_node->arguments->children.size() == 1
&& typeid_cast<const ASTAsterisk *>(func_node->arguments->children[0].get()))
{
func_node->arguments->children.clear();
}
}
String my_alias = ast->tryGetAlias();
if (!my_alias.empty())
data.current_alias = my_alias;
}
else if ((identifier_node = typeid_cast<ASTIdentifier *>(ast.get())))
{
if (identifier_node->general())
{
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
auto it_alias = aliases.find(identifier_node->name);
if (it_alias != aliases.end() && current_alias != identifier_node->name)
{
/// Let's replace it with the corresponding tree node.
if (current_asts.count(it_alias->second.get()))
throw Exception("Cyclic aliases", ErrorCodes::CYCLIC_ALIASES);
if (!my_alias.empty() && my_alias != it_alias->second->getAliasOrColumnName())
{
/// Avoid infinite recursion here
auto replace_to_identifier = typeid_cast<ASTIdentifier *>(it_alias->second.get());
bool is_cycle = replace_to_identifier && replace_to_identifier->general()
&& replace_to_identifier->name == identifier_node->name;
if (!is_cycle)
{
/// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a.
ast = it_alias->second->clone();
ast->setAlias(my_alias);
replaced = true;
}
}
else
{
ast = it_alias->second;
replaced = true;
}
}
}
}
else if (ASTExpressionList * expr_list = typeid_cast<ASTExpressionList *>(ast.get()))
{
/// Replace *, alias.*, database.table.* with a list of columns.
ASTs & asts = expr_list->children;
for (ssize_t expr_idx = asts.size() - 1; expr_idx >= 0; --expr_idx)
{
if (typeid_cast<const ASTAsterisk *>(asts[expr_idx].get()) && !all_column_names.empty())
{
asts.erase(asts.begin() + expr_idx);
for (size_t column_idx = 0; column_idx < all_column_names.size(); ++column_idx)
asts.insert(asts.begin() + column_idx + expr_idx, std::make_shared<ASTIdentifier>(all_column_names[column_idx]));
}
else if (typeid_cast<const ASTQualifiedAsterisk *>(asts[expr_idx].get()) && !table_names_and_column_names.empty())
{
const ASTQualifiedAsterisk * qualified_asterisk = static_cast<const ASTQualifiedAsterisk *>(asts[expr_idx].get());
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(qualified_asterisk->children[0].get());
size_t num_components = identifier->children.size();
for (const auto & [table_name, table_all_column_names] : table_names_and_column_names)
{
if ((num_components == 2 /// database.table.*
&& !table_name.database.empty() /// This is normal (not a temporary) table.
&& static_cast<const ASTIdentifier &>(*identifier->children[0]).name == table_name.database
&& static_cast<const ASTIdentifier &>(*identifier->children[1]).name == table_name.table)
|| (num_components == 0 /// t.*
&& ((!table_name.table.empty() && identifier->name == table_name.table) /// table.*
|| (!table_name.alias.empty() && identifier->name == table_name.alias)))) /// alias.*
{
asts.erase(asts.begin() + expr_idx);
for (size_t column_idx = 0; column_idx < table_all_column_names.size(); ++column_idx)
asts.insert(asts.begin() + column_idx + expr_idx, std::make_shared<ASTIdentifier>(table_all_column_names[column_idx]));
break;
}
}
}
}
}
else if (ASTTablesInSelectQueryElement * tables_elem = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
{
if (tables_elem->table_expression)
{
auto & database_and_table_name = static_cast<ASTTableExpression &>(*tables_elem->table_expression).database_and_table_name;
if (database_and_table_name)
{
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(database_and_table_name.get()))
right->setSpecial();
}
}
}
if (auto * node = typeid_cast<ASTFunction *>(ast.get()))
visit(*node, ast, data);
if (auto * node = typeid_cast<ASTIdentifier *>(ast.get()))
visit(*node, ast, data);
if (auto * node = typeid_cast<ASTExpressionList *>(ast.get()))
visit(*node, ast, data);
if (auto * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
visit(*node, ast, data);
if (auto * node = typeid_cast<ASTSelectQuery *>(ast.get()))
visit(*node, ast, data);
/// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias.
if (replaced)
{
performImpl(ast, finished_asts, current_asts, current_alias, level + 1);
current_asts.erase(initial_ast.get());
current_asts.erase(ast.get());
finished_asts[initial_ast] = ast;
return;
}
/// Recurring calls. Don't go into subqueries. Don't go into components of compound identifiers.
/// We also do not go to the left argument of lambda expressions, so as not to replace the formal parameters
/// on aliases in expressions of the form 123 AS x, arrayMap(x -> 1, [2]).
if (func_node && func_node->name == "lambda")
{
/// We skip the first argument. We also assume that the lambda function can not have parameters.
for (size_t i = 1, size = func_node->arguments->children.size(); i < size; ++i)
{
auto & child = func_node->arguments->children[i];
if (typeid_cast<const ASTSelectQuery *>(child.get()) || typeid_cast<const ASTTableExpression *>(child.get()))
continue;
performImpl(child, finished_asts, current_asts, current_alias, level + 1);
}
}
else if (identifier_node)
{
}
if (ast.get() != initial_ast.get())
visit(ast, data);
else
{
for (auto & child : ast->children)
{
if (typeid_cast<const ASTSelectQuery *>(child.get()) || typeid_cast<const ASTTableExpression *>(child.get()))
continue;
performImpl(child, finished_asts, current_asts, current_alias, level + 1);
}
}
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children, but also in where_expression and having_expression.
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(ast.get()))
{
if (select->prewhere_expression)
performImpl(select->prewhere_expression, finished_asts, current_asts, current_alias, level + 1);
if (select->where_expression)
performImpl(select->where_expression, finished_asts, current_asts, current_alias, level + 1);
if (select->having_expression)
performImpl(select->having_expression, finished_asts, current_asts, current_alias, level + 1);
}
visitChildren(ast, data);
current_asts.erase(initial_ast.get());
current_asts.erase(ast.get());
finished_asts[initial_ast] = ast;
/// @note can not place it in CheckASTDepth dtor cause of exception.
if (data.level == 1)
{
try
{
ast->checkSize(data.settings.max_expanded_ast_elements);
}
catch (Exception & e)
{
e.addMessage("(after expansion of aliases)");
throw;
}
}
}
}

View File

@ -18,8 +18,10 @@ inline bool functionIsInOrGlobalInOperator(const String & name)
}
using TableNameAndColumnNames = std::pair<DatabaseAndTableWithAlias, Names>;
using TableNamesAndColumnNames = std::vector<TableNameAndColumnNames>;
class ASTFunction;
class ASTIdentifier;
class ASTExpressionList;
struct ASTTablesInSelectQueryElement;
class QueryNormalizer
@ -41,23 +43,54 @@ class QueryNormalizer
public:
using Aliases = std::unordered_map<String, ASTPtr>;
using TableWithColumnNames = std::pair<DatabaseAndTableWithAlias, Names>;
QueryNormalizer(ASTPtr & query, const Aliases & aliases, ExtractedSettings && settings, const Names & all_columns_name,
const TableNamesAndColumnNames & table_names_and_column_names);
struct Data
{
using SetOfASTs = std::set<const IAST *>;
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
void perform();
const Aliases & aliases;
const ExtractedSettings settings;
const std::vector<TableWithColumnNames> tables_with_columns;
/// tmp data
size_t level;
MapOfASTs finished_asts; /// already processed vertices (and by what they replaced)
SetOfASTs current_asts; /// vertices in the current call stack of this method
std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
Data(const Aliases & aliases_, ExtractedSettings && settings_, std::vector<TableWithColumnNames> && tables_with_columns_ = {})
: aliases(aliases_)
, settings(settings_)
, tables_with_columns(tables_with_columns_)
, level(0)
{}
bool processAsterisks() const { return !tables_with_columns.empty(); }
};
QueryNormalizer(Data & data)
: visitor_data(data)
{}
void visit(ASTPtr & ast)
{
visit(ast, visitor_data);
}
private:
using SetOfASTs = std::set<const IAST *>;
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
Data & visitor_data;
ASTPtr & query;
const Aliases & aliases;
const ExtractedSettings settings;
const Names & all_column_names;
const TableNamesAndColumnNames & table_names_and_column_names;
static void visit(ASTPtr & query, Data & data);
void performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level);
static void visit(ASTIdentifier &, ASTPtr &, Data &);
static void visit(ASTFunction &, const ASTPtr &, Data &);
static void visit(ASTExpressionList &, const ASTPtr &, Data &);
static void visit(ASTTablesInSelectQueryElement &, const ASTPtr &, Data &);
static void visit(ASTSelectQuery &, const ASTPtr &, Data &);
static void visitChildren(const ASTPtr &, Data & data);
};
}

View File

@ -42,9 +42,9 @@ namespace ErrorCodes
extern const int INVALID_JOIN_ON_EXPRESSION;
}
void removeDuplicateColumns(NamesAndTypesList & columns)
NameSet removeDuplicateColumns(NamesAndTypesList & columns)
{
std::set<String> names;
NameSet names;
for (auto it = columns.begin(); it != columns.end();)
{
if (names.emplace(it->name).second)
@ -52,6 +52,7 @@ void removeDuplicateColumns(NamesAndTypesList & columns)
else
columns.erase(it++);
}
return names;
}
namespace
@ -77,8 +78,6 @@ void collectSourceColumns(ASTSelectQuery * select_query, StoragePtr storage, Nam
source_columns.insert(source_columns.end(), storage_aliases.begin(), storage_aliases.end());
}
}
removeDuplicateColumns(source_columns);
}
/// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names.
@ -102,12 +101,11 @@ void normalizeTree(
SyntaxAnalyzerResult & result,
const Names & source_columns,
const NameSet & source_columns_set,
const StoragePtr & storage,
const Context & context,
const ASTSelectQuery * select_query,
bool asterisk_left_columns_only)
{
Names all_columns_name = storage ? storage->getColumns().ordinary.getNames() : source_columns;
Names all_columns_name = source_columns;
if (!asterisk_left_columns_only)
{
@ -119,17 +117,20 @@ void normalizeTree(
if (all_columns_name.empty())
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR);
TableNamesAndColumnNames table_names_and_column_names;
std::vector<QueryNormalizer::TableWithColumnNames> table_with_columns;
if (select_query && select_query->tables && !select_query->tables->children.empty())
{
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
bool first = true;
String current_database = context.getCurrentDatabase();
for (const auto * table_expression : tables_expression)
{
DatabaseAndTableWithAlias table_name(*table_expression, context.getCurrentDatabase());
DatabaseAndTableWithAlias table_name(*table_expression, current_database);
NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context);
removeDuplicateColumns(names_and_types);
if (!first)
{
/// For joined tables qualify duplicating names.
@ -140,12 +141,14 @@ void normalizeTree(
first = false;
table_names_and_column_names.emplace_back(std::pair(table_name, names_and_types.getNames()));
table_with_columns.emplace_back(std::move(table_name), names_and_types.getNames());
}
}
else
table_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns_name));
auto & settings = context.getSettingsRef();
QueryNormalizer(query, result.aliases, settings, all_columns_name, table_names_and_column_names).perform();
QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef(), std::move(table_with_columns));
QueryNormalizer(normalizer_data).visit(query);
}
bool hasArrayJoin(const ASTPtr & ast)
@ -440,7 +443,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
const String nested_table_name = ast->getColumnName();
const String nested_table_alias = ast->getAliasOrColumnName();
if (nested_table_alias == nested_table_name && !typeid_cast<const ASTIdentifier *>(ast.get()))
if (nested_table_alias == nested_table_name && !isIdentifier(ast))
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name,
ErrorCodes::ALIAS_REQUIRED);
@ -468,7 +471,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
String result_name = expr->getAliasOrColumnName();
/// This is an array.
if (!typeid_cast<ASTIdentifier *>(expr.get()) || source_columns_set.count(source_name))
if (!isIdentifier(expr) || source_columns_set.count(source_name))
{
result.array_join_result_to_source[result_name] = source_name;
}
@ -525,10 +528,10 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS
std::function<TableBelonging(const ASTPtr &)> get_table_belonging;
get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging
{
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
if (identifier)
if (getColumnIdentifierName(ast))
{
if (identifier->general())
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
{
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names);
@ -564,9 +567,10 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS
std::function<void(ASTPtr &, const DatabaseAndTableWithAlias &, bool)> translate_qualified_names;
translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table)
{
if (auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get()))
if (getColumnIdentifierName(ast))
{
if (identifier->general())
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
{
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
stripIdentifier(ast, num_components);
@ -739,6 +743,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
result.source_columns = source_columns_;
collectSourceColumns(select_query, result.storage, result.source_columns);
NameSet source_columns_set = removeDuplicateColumns(result.source_columns);
const auto & settings = context.getSettingsRef();
@ -746,7 +751,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
source_columns_list.reserve(result.source_columns.size());
for (const auto & type_name : result.source_columns)
source_columns_list.emplace_back(type_name.name);
NameSet source_columns_set(source_columns_list.begin(), source_columns_list.end());
if (source_columns_set.size() != source_columns_list.size())
throw Exception("Unexpected duplicates in source columns list.", ErrorCodes::LOGICAL_ERROR);
if (select_query)
{
@ -768,7 +775,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
}
/// Common subexpression elimination. Rewrite rules.
normalizeTree(query, result, source_columns_list, source_columns_set, result.storage,
normalizeTree(query, result, (storage ? storage->getColumns().ordinary.getNames() : source_columns_list), source_columns_set,
context, select_query, settings.asterisk_left_columns_only != 0);
/// Remove unneeded columns according to 'required_result_columns'.

View File

@ -8,6 +8,8 @@ namespace DB
class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
struct SyntaxAnalyzerResult
{
StoragePtr storage;

View File

@ -55,7 +55,7 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(const ASTIdentifier
const NameSet & source_columns = data.source_columns;
const std::vector<DatabaseAndTableWithAlias> & tables = data.tables;
if (identifier.general())
if (getColumnIdentifierName(identifier))
{
/// Select first table name with max number of qualifiers which can be stripped.
size_t max_num_qualifiers_to_strip = 0;

View File

@ -499,7 +499,7 @@ void executeQuery(
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->format).name
? *getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat();
BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader());

View File

@ -22,6 +22,7 @@
#include <Common/escapeForFileName.h>
#include <Common/Stopwatch.h>
#include <Common/typeid_cast.h>
namespace DB

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
@ -38,4 +39,82 @@ void ASTIdentifier::appendColumnNameImpl(WriteBuffer & ostr) const
writeString(name, ostr);
}
ASTPtr createTableIdentifier(const String & database_name, const String & table_name)
{
if (database_name.empty())
return ASTIdentifier::createSpecial(table_name);
ASTPtr database = ASTIdentifier::createSpecial(database_name);
ASTPtr table = ASTIdentifier::createSpecial(table_name);
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
database_and_table->children = {database, table};
return database_and_table;
}
bool isIdentifier(const IAST * const ast)
{
if (ast)
return typeid_cast<const ASTIdentifier *>(ast);
return false;
}
std::optional<String> getIdentifierName(const IAST * const ast)
{
if (ast)
if (auto node = typeid_cast<const ASTIdentifier *>(ast))
return node->name;
return {};
}
bool getIdentifierName(const ASTPtr & ast, String & name)
{
if (ast)
if (auto node = typeid_cast<const ASTIdentifier *>(ast.get()))
{
name = node->name;
return true;
}
return false;
}
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node)
{
if (!node.special)
return node.name;
return {};
}
std::optional<String> getColumnIdentifierName(const ASTPtr & ast)
{
if (ast)
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
if (!id->special)
return id->name;
return {};
}
std::optional<String> getTableIdentifierName(const ASTIdentifier & node)
{
if (node.special)
return node.name;
return {};
}
std::optional<String> getTableIdentifierName(const ASTPtr & ast)
{
if (ast)
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
if (id->special)
return id->name;
return {};
}
void setIdentifierSpecial(ASTPtr & ast)
{
if (ast)
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(ast.get()))
id->setSpecial();
}
}

View File

@ -1,27 +1,26 @@
#pragma once
#include <optional>
#include <Parsers/ASTWithAlias.h>
namespace DB
{
/** Identifier (column or alias)
*/
/// Identifier (column, table or alias)
class ASTIdentifier : public ASTWithAlias
{
enum Kind /// TODO This is semantic, not syntax. Remove it.
{
General,
Special, // Database, Table, Format
};
public:
/// name. The composite identifier here will have a concatenated name (of the form a.b.c), and individual components will be available inside the children.
String name;
ASTIdentifier(const String & name_, const Kind kind_ = General)
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); }
ASTIdentifier(const String & name_)
: name(name_)
, special(false)
{
range = StringRange(name.data(), name.data() + name.size());
}
/** Get the text that identifies this element. */
String getID(char delim) const override { return "Identifier" + (delim + name); }
@ -33,21 +32,50 @@ public:
set.insert(name);
}
void setSpecial() { kind = Special; }
bool general() const { return kind == General; }
bool special() const { return kind == Special; }
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
{
return std::make_shared<ASTIdentifier>(name_, ASTIdentifier::Special);
}
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
private:
Kind kind;
bool special; /// TODO: it would be ptr to semantic here
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
{
auto ret = std::make_shared<ASTIdentifier>(name_);
ret->special = true;
return ret;
}
void setSpecial() { special = true; }
friend void setIdentifierSpecial(ASTPtr &);
friend std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
friend std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
friend std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
friend std::optional<String> getTableIdentifierName(const ASTPtr & ast);
friend ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
};
/// ASTIdentifier Helpers: hide casts and semantic.
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
bool isIdentifier(const IAST * const ast);
inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); }
std::optional<String> getIdentifierName(const IAST * const ast);
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
bool getIdentifierName(const ASTPtr & ast, String & name);
/// @returns name for column identifiers
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
/// @returns name for 'not a column' identifiers
std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
std::optional<String> getTableIdentifierName(const ASTPtr & ast);
void setIdentifierSpecial(ASTPtr & ast);
}

View File

@ -17,19 +17,6 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
ASTPtr createDatabaseAndTableNode(const String & database_name, const String & table_name)
{
if (database_name.empty())
return ASTIdentifier::createSpecial(table_name);
ASTPtr database = ASTIdentifier::createSpecial(database_name);
ASTPtr table = ASTIdentifier::createSpecial(table_name);
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
database_and_table->children = {database, table};
return database_and_table;
}
ASTPtr ASTSelectQuery::clone() const
{
@ -338,7 +325,7 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
}
String table_alias = getTableExpressionAlias(table_expression);
table_expression->database_and_table_name = createDatabaseAndTableNode(database_name, table_name);
table_expression->database_and_table_name = createTableIdentifier(database_name, table_name);
if (!table_alias.empty())
table_expression->database_and_table_name->setAlias(table_alias);

View File

@ -52,7 +52,4 @@ protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
ASTPtr createDatabaseAndTableNode(const String & database_name, const String & table_name);
}

View File

@ -5,6 +5,7 @@
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromMemory.h>
#include <Common/typeid_cast.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/IAST.h>
@ -173,7 +174,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
{
if (!name.empty())
name += '.';
name += static_cast<const ASTIdentifier &>(*child.get()).name;
name += *getIdentifierName(child);
}
node = std::make_shared<ASTIdentifier>(name);
@ -222,7 +223,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
* If you do not report that the first option is an error, then the argument will be interpreted as 2014 - 01 - 01 - some number,
* and the query silently returns an unexpected result.
*/
if (typeid_cast<const ASTIdentifier &>(*identifier).name == "toDate"
if (*getIdentifierName(identifier) == "toDate"
&& contents_end - contents_begin == strlen("2014-01-01")
&& contents_begin[0] >= '2' && contents_begin[0] <= '3'
&& contents_begin[1] >= '0' && contents_begin[1] <= '9'
@ -264,7 +265,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
auto function_node = std::make_shared<ASTFunction>();
function_node->name = typeid_cast<ASTIdentifier &>(*identifier).name;
getIdentifierName(identifier, function_node->name);
/// func(DISTINCT ...) is equivalent to funcDistinct(...)
if (has_distinct_modifier)
@ -1157,7 +1158,7 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
* and in the query "SELECT x FRO FROM t", the word FRO was considered an alias.
*/
const String & name = static_cast<const ASTIdentifier &>(*node.get()).name;
const String name = *getIdentifierName(node);
for (const char ** keyword = restricted_keywords; *keyword != nullptr; ++keyword)
if (0 == strcasecmp(name.data(), *keyword))
@ -1249,18 +1250,16 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
*/
bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword;
if (allow_alias_without_as_keyword)
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(node.get()))
if (0 == strcasecmp(id->name.data(), "FROM"))
if (auto opt_id = getIdentifierName(node))
if (0 == strcasecmp(opt_id->data(), "FROM"))
allow_alias_without_as_keyword_now = false;
ASTPtr alias_node;
if (ParserAlias(allow_alias_without_as_keyword_now).parse(pos, alias_node, expected))
{
String alias_name = typeid_cast<const ASTIdentifier &>(*alias_node).name;
if (ASTWithAlias * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get()))
{
ast_with_alias->alias = alias_name;
getIdentifierName(alias_node, ast_with_alias->alias);
ast_with_alias->prefer_alias_to_column_name = prefer_alias_to_column_name;
}
else

View File

@ -1,10 +1,9 @@
#include <Common/typeid_cast.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>

View File

@ -1,3 +1,4 @@
#include <Common/typeid_cast.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
@ -312,7 +313,7 @@ bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!p_expression.parse(pos, assignment->expression, expected))
return false;
assignment->column_name = typeid_cast<const ASTIdentifier &>(*column).name;
getIdentifierName(column, assignment->column_name);
if (assignment->expression)
assignment->children.push_back(assignment->expression);

View File

@ -4,8 +4,6 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTCheckQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -31,15 +29,15 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
auto query = std::make_shared<ASTCheckQuery>();
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
node = query;
}
else
{
table = database;
auto query = std::make_shared<ASTCheckQuery>();
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
getIdentifierName(table, query->table);
node = query;
}

View File

@ -35,7 +35,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
auto func = std::make_shared<ASTFunction>();
func->name = typeid_cast<ASTIdentifier &>(*name).name;
getIdentifierName(name, func->name);
func->arguments = columns;
func->children.push_back(columns);
node = func;
@ -70,7 +70,7 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
if (non_parametric.parse(pos, ident, expected))
{
auto func = std::make_shared<ASTFunction>();
func->name = typeid_cast<ASTIdentifier &>(*ident).name;
getIdentifierName(ident, func->name);
node = func;
return true;
}
@ -257,10 +257,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->if_not_exists = if_not_exists;
query->cluster = cluster_str;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
return true;
}
@ -405,23 +403,18 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->is_populate = is_populate;
query->temporary = is_temporary;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
query->cluster = cluster_str;
if (to_database)
query->to_database = typeid_cast<ASTIdentifier &>(*to_database).name;
if (to_table)
query->to_table = typeid_cast<ASTIdentifier &>(*to_table).name;
getIdentifierName(to_database, query->to_database);
getIdentifierName(to_table, query->to_table);
query->set(query->columns, columns);
query->set(query->storage, storage);
if (as_database)
query->as_database = typeid_cast<ASTIdentifier &>(*as_database).name;
if (as_table)
query->as_table = typeid_cast<ASTIdentifier &>(*as_table).name;
getIdentifierName(as_database, query->as_database);
getIdentifierName(as_table, query->as_table);
query->set(query->select, select);
return true;

View File

@ -8,7 +8,6 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/CommonParsers.h>
#include <Common/typeid_cast.h>
#include <Poco/String.h>
@ -74,7 +73,7 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expect
&& type_parser.parse(pos, type, expected))
{
auto name_type_pair = std::make_shared<ASTNameTypePair>();
name_type_pair->name = typeid_cast<const ASTIdentifier &>(*name).name;
getIdentifierName(name, name_type_pair->name);
name_type_pair->type = type;
name_type_pair->children.push_back(type);
node = name_type_pair;
@ -181,7 +180,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
node = column_declaration;
column_declaration->name = typeid_cast<ASTIdentifier &>(*name).name;
getIdentifierName(name, column_declaration->name);
if (type)
{

View File

@ -4,8 +4,6 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/ParserDropQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -117,10 +115,10 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
query->kind = ASTDropQuery::Kind::Drop;
query->if_exists = if_exists;
query->temporary = temporary;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
query->cluster = cluster_str;
return true;

View File

@ -9,8 +9,6 @@
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -136,14 +134,11 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else
{
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
}
if (format)
query->format = typeid_cast<ASTIdentifier &>(*format).name;
getIdentifierName(format, query->format);
query->columns = columns;
query->select = select;

View File

@ -5,8 +5,6 @@
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -60,10 +58,8 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
auto query = std::make_shared<ASTOptimizeQuery>();
node = query;
if (database)
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
query->cluster = cluster_str;
query->partition = partition;

View File

@ -76,7 +76,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!format_p.parse(pos, query_with_output.format, expected))
return false;
typeid_cast<ASTIdentifier &>(*(query_with_output.format)).setSpecial();
setIdentifierSpecial(query_with_output.format);
query_with_output.children.push_back(query_with_output.format);
}

View File

@ -4,8 +4,6 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/ParserRenameQuery.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -31,8 +29,9 @@ static bool parseDatabaseAndTable(
return false;
}
db_and_table.database = database ? typeid_cast<const ASTIdentifier &>(*database).name : "";
db_and_table.table = typeid_cast<const ASTIdentifier &>(*table).name;
db_and_table.database.clear();
getIdentifierName(database, db_and_table.database);
getIdentifierName(table, db_and_table.table);
return true;
}

View File

@ -31,7 +31,7 @@ static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos,
if (!value_p.parse(pos, value, expected))
return false;
change.name = typeid_cast<const ASTIdentifier &>(*name).name;
getIdentifierName(name, change.name);
change.value = typeid_cast<const ASTLiteral &>(*value).value;
return true;

View File

@ -65,8 +65,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
if (database)
query->from = typeid_cast<ASTIdentifier &>(*database).name;
getIdentifierName(database, query->from);
if (like)
query->like = safeGet<const String &>(typeid_cast<ASTLiteral &>(*like).value);

View File

@ -75,10 +75,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
}
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);
node = query;

View File

@ -15,16 +15,15 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_use("USE");
ParserIdentifier name_p;
ASTPtr database;
if (!s_use.ignore(pos, expected))
return false;
ASTPtr database;
if (!name_p.parse(pos, database, expected))
return false;
auto query = std::make_shared<ASTUseQuery>();
query->database = typeid_cast<ASTIdentifier &>(*database).name;
getIdentifierName(database, query->database);
node = query;
return true;

View File

@ -2,7 +2,6 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Common/typeid_cast.h>
namespace DB
@ -30,13 +29,13 @@ bool parseDatabaseAndTableName(IParser::Pos & pos, Expected & expected, String &
return false;
}
database_str = typeid_cast<ASTIdentifier &>(*database).name;
table_str = typeid_cast<ASTIdentifier &>(*table).name;
getIdentifierName(database, database_str);
getIdentifierName(table, table_str);
}
else
{
database_str = "";
table_str = typeid_cast<ASTIdentifier &>(*database).name;
getIdentifierName(database, table_str);
}
return true;

View File

@ -20,7 +20,7 @@ bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, Str
result = typeid_cast<const ASTLiteral &>(*res).value.safeGet<String>();
}
else
result = typeid_cast<const ASTIdentifier &>(*res).name;
result = *getIdentifierName(res);
return true;
}

View File

@ -14,6 +14,7 @@
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Common/typeid_cast.h>
#include <Compression/CompressionFactory.h>
namespace DB
@ -30,6 +31,7 @@ namespace ErrorCodes
std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast)
{
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
const CompressionCodecFactory & compression_codec_factory = CompressionCodecFactory::instance();
if (command_ast->type == ASTAlterCommand::ADD_COLUMN)
{
@ -49,8 +51,11 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.default_expression = ast_col_decl.default_expression;
}
if (ast_col_decl.codec)
command.codec = compression_codec_factory.get(ast_col_decl.codec);
if (command_ast->column)
command.after_column = typeid_cast<const ASTIdentifier &>(*command_ast->column).name;
command.after_column = *getIdentifierName(command_ast->column);
command.if_not_exists = command_ast->if_not_exists;
@ -63,7 +68,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
AlterCommand command;
command.type = AlterCommand::DROP_COLUMN;
command.column_name = typeid_cast<const ASTIdentifier &>(*(command_ast->column)).name;
command.column_name = *getIdentifierName(command_ast->column);
command.if_exists = command_ast->if_exists;
return command;
}
@ -86,6 +91,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.default_expression = ast_col_decl.default_expression;
}
if (ast_col_decl.codec)
command.codec = compression_codec_factory.get(ast_col_decl.codec);
if (ast_col_decl.comment)
{
const auto & ast_comment = typeid_cast<ASTLiteral &>(*ast_col_decl.comment);
@ -99,8 +107,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
{
AlterCommand command;
command.type = COMMENT_COLUMN;
const auto & ast_identifier = typeid_cast<ASTIdentifier &>(*command_ast->column);
command.column_name = ast_identifier.name;
command.column_name = *getIdentifierName(command_ast->column);
const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment);
command.comment = ast_comment.value.get<String>();
command.if_exists = command_ast->if_exists;
@ -169,6 +176,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
if (default_expression)
columns_description.defaults.emplace(column_name, ColumnDefault{default_kind, default_expression});
if (codec)
columns_description.codecs.emplace(column_name, codec);
/// Slow, because each time a list is copied
columns_description.ordinary = Nested::flatten(columns_description.ordinary);
}
@ -201,6 +211,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
}
else if (type == MODIFY_COLUMN)
{
if (codec)
columns_description.codecs[column_name] = codec;
if (!is_mutable())
{
auto & comments = columns_description.comments;

View File

@ -55,6 +55,9 @@ struct AlterCommand
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
bool ignore = false;
/// For ADD and MODIFY
CompressionCodecPtr codec;
AlterCommand() = default;
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
const ColumnDefaultKind default_kind, const ASTPtr & default_expression,

View File

@ -1,40 +1,34 @@
#include <Common/config.h>
#include <Common/config_version.h>
#include <Storages/Kafka/StorageKafka.h>
#if USE_RDKAFKA
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Macros.h>
#include <Common/Exception.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <Formats/FormatFactory.h>
#include <IO/ReadBuffer.h>
#include <Interpreters/Context.h>
#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 <Parsers/ASTCreateQuery.h>
#include <Storages/Kafka/KafkaSettings.h>
#include <Storages/Kafka/StorageKafka.h>
#include <Storages/StorageMaterializedView.h>
#include <Storages/StorageFactory.h>
#include <IO/ReadBuffer.h>
#include <Storages/StorageMaterializedView.h>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Exception.h>
#include <Common/Macros.h>
#include <Common/config_version.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
#include <common/logger_useful.h>
#if __has_include(<rdkafka.h>) // maybe bundled
#include <rdkafka.h> // Y_IGNORE
#else // system
#include <librdkafka/rdkafka.h>
#endif
namespace DB
{
@ -62,8 +56,8 @@ static const String CONFIG_PREFIX = "kafka";
class ReadBufferFromKafkaConsumer : public ReadBuffer
{
rd_kafka_t * consumer;
rd_kafka_message_t * current = nullptr;
ConsumerPtr consumer;
cppkafka::Message current;
bool current_pending = false; /// We've fetched "current" message and need to process it on the next iteration.
Poco::Logger * log;
size_t read_messages = 0;
@ -73,42 +67,36 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer
{
if (current_pending)
{
BufferBase::set(reinterpret_cast<char *>(current->payload), current->len, 0);
// XXX: very fishy place with const casting.
BufferBase::set(reinterpret_cast<char *>(const_cast<unsigned char *>(current.get_payload().get_data())), current.get_payload().get_size(), 0);
current_pending = false;
return true;
}
// Process next buffered message
rd_kafka_message_t * msg = rd_kafka_consumer_poll(consumer, READ_POLL_MS); // XXX: use RAII.
if (msg == nullptr)
auto message = consumer->poll(std::chrono::milliseconds(READ_POLL_MS));
if (!message)
return false;
if (msg->err)
if (message.is_eof())
{
if (msg->err != RD_KAFKA_RESP_ERR__PARTITION_EOF)
{
LOG_ERROR(log, "Consumer error: " << rd_kafka_err2str(msg->err) << " " << rd_kafka_message_errstr(msg));
rd_kafka_message_destroy(msg);
return false;
}
// Reach EOF while reading current batch, skip it
LOG_TRACE(log, "EOF reached for partition " << msg->partition << " offset " << msg->offset);
rd_kafka_message_destroy(msg);
// Reached EOF while reading current batch, skip it.
LOG_TRACE(log, "EOF reached for partition " << message.get_partition() << " offset " << message.get_offset());
return nextImpl();
}
if (msg->len && !msg->payload)
throw Exception("Logical error: nullptr message returned with non-zero length", ErrorCodes::LOGICAL_ERROR);
else if (auto err = message.get_error())
{
LOG_ERROR(log, "Consumer error: " << err);
return false;
}
++read_messages;
// Now we've received a new message. Check if we need to produce a delimiter
if (row_delimiter != '\0' && current != nullptr)
if (row_delimiter != '\0' && current)
{
BufferBase::set(&row_delimiter, 1, 0);
reset();
current = msg;
current = std::move(message);
current_pending = true;
return true;
}
@ -116,31 +104,21 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer
// Consume message and mark the topic/partition offset
// The offsets will be committed in the readSuffix() method after the block is completed
// If an exception is thrown before that would occur, the client will rejoin without committing offsets
reset();
current = msg;
BufferBase::set(reinterpret_cast<char *>(current->payload), current->len, 0);
current = std::move(message);
// XXX: very fishy place with const casting.
BufferBase::set(reinterpret_cast<char *>(const_cast<unsigned char *>(current.get_payload().get_data())), current.get_payload().get_size(), 0);
return true;
}
void reset()
{
if (current != nullptr)
{
rd_kafka_message_destroy(current);
current = nullptr;
}
}
public:
ReadBufferFromKafkaConsumer(rd_kafka_t * consumer_, Poco::Logger * log_, char row_delimiter_)
ReadBufferFromKafkaConsumer(ConsumerPtr consumer_, Poco::Logger * log_, char row_delimiter_)
: ReadBuffer(nullptr, 0), consumer(consumer_), log(log_), row_delimiter(row_delimiter_)
{
if (row_delimiter != '\0')
LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
}
~ReadBufferFromKafkaConsumer() override { reset(); }
/// Commit messages read with this consumer
void commit()
{
@ -148,10 +126,7 @@ public:
if (read_messages == 0)
return;
auto err = rd_kafka_commit(consumer, nullptr, 1 /* async */);
if (err)
throw Exception("Failed to commit offsets: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
consumer->async_commit();
read_messages = 0;
}
};
@ -215,7 +190,7 @@ public:
if (consumer == nullptr)
throw Exception("Failed to claim consumer: ", ErrorCodes::TIMEOUT_EXCEEDED);
read_buf = std::make_unique<ReadBufferFromKafkaConsumer>(consumer->stream, storage.log, storage.row_delimiter);
read_buf = std::make_unique<ReadBufferFromKafkaConsumer>(consumer, storage.log, storage.row_delimiter);
reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size);
}
@ -239,7 +214,7 @@ public:
private:
StorageKafka & storage;
StorageKafka::ConsumerPtr consumer;
ConsumerPtr consumer;
Context context;
size_t max_block_size;
Block sample_block;
@ -251,7 +226,7 @@ private:
bool hasClaimed() { return consumer != nullptr; }
};
static void loadFromConfig(struct rd_kafka_conf_s * conf, const AbstractConfiguration & config, const std::string & path)
static void loadFromConfig(cppkafka::Configuration & conf, const AbstractConfiguration & config, const std::string & path)
{
AbstractConfiguration::Keys keys;
std::vector<char> errstr(512);
@ -262,8 +237,7 @@ static void loadFromConfig(struct rd_kafka_conf_s * conf, const AbstractConfigur
{
const String key_path = path + "." + key;
const String key_name = boost::replace_all_copy(key, "_", ".");
if (rd_kafka_conf_set(conf, key_name.c_str(), config.getString(key_path).c_str(), errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
throw Exception("Invalid Kafka setting " + key_path + " in config: " + String(errstr.data()), ErrorCodes::INVALID_CONFIG_PARAMETER);
conf.set(key_name, config.getString(key_path));
}
}
@ -326,21 +300,8 @@ void StorageKafka::startup()
{
for (size_t i = 0; i < num_consumers; ++i)
{
// Building configuration may throw, the consumer configuration must be destroyed in that case
auto consumer_conf = rd_kafka_conf_new();
try
{
consumerConfiguration(consumer_conf);
}
catch (...)
{
rd_kafka_conf_destroy(consumer_conf);
throw;
}
// Create a consumer and subscribe to topics
// Note: consumer takes ownership of the configuration
auto consumer = std::make_shared<StorageKafka::Consumer>(consumer_conf);
auto consumer = std::make_shared<cppkafka::Consumer>(createConsumerConfiguration());
consumer->subscribe(topics);
// Make consumer available
@ -362,7 +323,7 @@ void StorageKafka::shutdown()
for (size_t i = 0; i < num_created_consumers; ++i)
{
auto consumer = claimConsumer();
consumer->close();
// FIXME: not sure if really close consumers here, and if we really need to close them here.
}
LOG_TRACE(log, "Waiting for cleanup");
@ -378,24 +339,20 @@ void StorageKafka::updateDependencies()
}
void StorageKafka::consumerConfiguration(struct rd_kafka_conf_s * conf)
cppkafka::Configuration StorageKafka::createConsumerConfiguration()
{
std::vector<char> errstr(512);
cppkafka::Configuration conf;
LOG_TRACE(log, "Setting brokers: " << brokers);
if (rd_kafka_conf_set(conf, "metadata.broker.list", brokers.c_str(), errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
conf.set("metadata.broker.list", brokers);
LOG_TRACE(log, "Setting Group ID: " << group << " Client ID: clickhouse");
conf.set("group.id", group);
if (rd_kafka_conf_set(conf, "group.id", group.c_str(), errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
if (rd_kafka_conf_set(conf, "client.id", VERSION_FULL, errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
conf.set("client.id", VERSION_FULL);
// We manually commit offsets after a stream successfully finished
rd_kafka_conf_set(conf, "enable.auto.commit", "false", nullptr, 0);
conf.set("enable.auto.commit", "false");
// Update consumer configuration from the configuration
const auto & config = global_context.getConfigRef();
@ -409,14 +366,16 @@ void StorageKafka::consumerConfiguration(struct rd_kafka_conf_s * conf)
if (config.has(topic_config_key))
loadFromConfig(conf, config, topic_config_key);
}
return conf;
}
StorageKafka::ConsumerPtr StorageKafka::claimConsumer()
ConsumerPtr StorageKafka::claimConsumer()
{
return tryClaimConsumer(-1L);
}
StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
{
// Wait for the first free consumer
if (wait_ms >= 0)
@ -434,7 +393,7 @@ StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
return consumer;
}
void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr consumer)
void StorageKafka::pushConsumer(ConsumerPtr consumer)
{
std::lock_guard lock(mutex);
consumers.push_back(consumer);
@ -557,64 +516,6 @@ bool StorageKafka::streamToViews()
}
StorageKafka::Consumer::Consumer(struct rd_kafka_conf_s * conf)
{
std::vector<char> errstr(512);
stream = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr.data(), errstr.size());
if (stream == nullptr)
{
rd_kafka_conf_destroy(conf);
throw Exception("Failed to create consumer handle: " + String(errstr.data()), ErrorCodes::UNKNOWN_EXCEPTION);
}
rd_kafka_poll_set_consumer(stream);
}
StorageKafka::Consumer::~Consumer()
{
close();
}
void StorageKafka::Consumer::subscribe(const Names & topics_to_subscribe)
{
if (stream == nullptr)
throw Exception("Cannot subscribe to topics when consumer is closed", ErrorCodes::UNKNOWN_EXCEPTION);
// Create a list of partitions
auto * topic_list = rd_kafka_topic_partition_list_new(topics_to_subscribe.size());
for (const auto & topic : topics_to_subscribe)
rd_kafka_topic_partition_list_add(topic_list, topic.c_str(), RD_KAFKA_PARTITION_UA);
// Subscribe to requested topics
auto err = rd_kafka_subscribe(stream, topic_list);
if (err)
{
rd_kafka_topic_partition_list_destroy(topic_list);
throw Exception("Failed to subscribe: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
}
rd_kafka_topic_partition_list_destroy(topic_list);
}
void StorageKafka::Consumer::unsubscribe()
{
if (stream != nullptr)
rd_kafka_unsubscribe(stream);
}
void StorageKafka::Consumer::close()
{
if (stream != nullptr)
{
rd_kafka_consumer_close(stream);
rd_kafka_destroy(stream);
stream = nullptr;
}
}
void registerStorageKafka(StorageFactory & factory)
{
factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args)

View File

@ -1,24 +1,24 @@
#pragma once
#include <Common/config.h>
#if USE_RDKAFKA
#include <mutex>
#include <ext/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Core/NamesAndTypes.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Storages/IStorage.h>
#include <Poco/Event.h>
#include <Poco/Semaphore.h>
#include <ext/shared_ptr_helper.h>
struct rd_kafka_s;
struct rd_kafka_conf_s;
#include <cppkafka/cppkafka.h>
#include <mutex>
namespace DB
{
class StorageKafka;
using ConsumerPtr = std::shared_ptr<cppkafka::Consumer>;
/** Implements a Kafka queue table engine that can be used as a persistent queue / buffer,
* or as a basic building block for creating pipelines with a continuous insertion / ETL.
@ -53,22 +53,6 @@ public:
void updateDependencies() override;
private:
/// Each engine typically has one consumer (able to process 1..N partitions)
/// It is however possible to create multiple consumers per table, as long
/// as the total number of consumers is <= number of partitions.
struct Consumer
{
Consumer(struct rd_kafka_conf_s * conf);
~Consumer();
void subscribe(const Names & topics);
void unsubscribe();
void close();
struct rd_kafka_s * stream = nullptr;
};
using ConsumerPtr = std::shared_ptr<Consumer>;
// Configuration and state
String table_name;
String database_name;
@ -100,7 +84,7 @@ private:
BackgroundSchedulePool::TaskHolder task;
std::atomic<bool> stream_cancelled{false};
void consumerConfiguration(struct rd_kafka_conf_s * conf);
cppkafka::Configuration createConsumerConfiguration();
ConsumerPtr claimConsumer();
ConsumerPtr tryClaimConsumer(long wait_ms);
void pushConsumer(ConsumerPtr c);

View File

@ -121,7 +121,7 @@ void MergeTreeWhereOptimizer::optimizeConjunction(ASTSelectQuery & select, ASTFu
SCOPE_EXIT(++idx);
if (cannotBeMoved(condition))
if (cannotBeMoved(conditions[idx]))
continue;
IdentifierNameSet identifiers{};
@ -193,7 +193,7 @@ void MergeTreeWhereOptimizer::optimizeArbitrary(ASTSelectQuery & select) const
auto & condition = select.where_expression;
/// do not optimize restricted expressions
if (cannotBeMoved(select.where_expression.get()))
if (cannotBeMoved(select.where_expression))
return;
IdentifierNameSet identifiers{};
@ -250,10 +250,10 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
auto right_arg = function->arguments->children.back().get();
/// try to ensure left_arg points to ASTIdentifier
if (!typeid_cast<const ASTIdentifier *>(left_arg) && typeid_cast<const ASTIdentifier *>(right_arg))
if (!isIdentifier(left_arg) && isIdentifier(right_arg))
std::swap(left_arg, right_arg);
if (typeid_cast<const ASTIdentifier *>(left_arg))
if (isIdentifier(left_arg))
{
/// condition may be "good" if only right_arg is a constant and its value is outside the threshold
if (const auto literal = typeid_cast<const ASTLiteral *>(right_arg))
@ -286,8 +286,8 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
void MergeTreeWhereOptimizer::collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set)
{
if (const auto identifier = typeid_cast<const ASTIdentifier *>(ast))
return (void) set.insert(identifier->name);
if (auto opt_name = getIdentifierName(ast))
return (void) set.insert(*opt_name);
if (typeid_cast<const ASTSubquery *>(ast))
return;
@ -364,9 +364,9 @@ bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const IdentifierNameSet & i
}
bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const
{
if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr))
if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr.get()))
{
/// disallow arrayJoin expressions to be moved to PREWHERE for now
if (array_join_function_name == function_ptr->name)
@ -381,17 +381,16 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
if ("indexHint" == function_ptr->name)
return true;
}
else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr))
else if (auto opt_name = getColumnIdentifierName(ptr))
{
/// disallow moving result of ARRAY JOIN to PREWHERE
if (identifier_ptr->general())
if (array_joined_names.count(identifier_ptr->name) ||
array_joined_names.count(Nested::extractTableName(identifier_ptr->name)))
return true;
if (array_joined_names.count(*opt_name) ||
array_joined_names.count(Nested::extractTableName(*opt_name)))
return true;
}
for (const auto & child : ptr->children)
if (cannotBeMoved(child.get()))
if (cannotBeMoved(child))
return true;
return false;

View File

@ -68,7 +68,7 @@ private:
*
* Also, disallow moving expressions with GLOBAL [NOT] IN.
*/
bool cannotBeMoved(const IAST * ptr) const;
bool cannotBeMoved(const ASTPtr & ptr) const;
void determineArrayJoinedNames(ASTSelectQuery & select);

View File

@ -295,7 +295,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
/// Virtual node, all nodes that are "greater" than this one will be deleted
NodeWithStat block_threshold{{}, time_threshold};
size_t current_deduplication_window = std::min(timed_blocks.size(), storage.data.settings.replicated_deduplication_window.value);
size_t current_deduplication_window = std::min<size_t>(timed_blocks.size(), storage.data.settings.replicated_deduplication_window.value);
auto first_outdated_block_fixed_threshold = timed_blocks.begin() + current_deduplication_window;
auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime);
auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold);

View File

@ -42,13 +42,13 @@ static Names extractColumnNames(const ASTPtr & node)
Names res;
res.reserve(elements.size());
for (const auto & elem : elements)
res.push_back(typeid_cast<const ASTIdentifier &>(*elem).name);
res.push_back(*getIdentifierName(elem));
return res;
}
else
{
return { typeid_cast<const ASTIdentifier &>(*node).name };
return { *getIdentifierName(node) };
}
}
@ -481,9 +481,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
{
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
merging_params.sign_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
@ -495,9 +493,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// If the last element is not index_granularity or replica_name (a literal), then this is the name of the version column.
if (!engine_args.empty() && !typeid_cast<const ASTLiteral *>(engine_args.back().get()))
{
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
merging_params.version_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.version_column))
throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
@ -535,18 +531,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
}
else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
{
if (auto ast = typeid_cast<ASTIdentifier *>(engine_args.back().get()))
merging_params.version_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.version_column))
throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
engine_args.pop_back();
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
merging_params.sign_column = ast->name;
else
if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
@ -592,9 +584,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// Now only three parameters remain - date (or partitioning expression), primary_key, index_granularity.
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args[0].get()))
date_column_name = ast->name;
else
if (!getIdentifierName(engine_args[0], date_column_name))
throw Exception(
"Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);

View File

@ -2,7 +2,6 @@
#include <Storages/IStorage.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
namespace DB
@ -66,7 +65,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
PartitionCommand res;
res.type = CLEAR_COLUMN;
res.partition = command_ast->partition;
const Field & column_name = typeid_cast<const ASTIdentifier &>(*(command_ast->column)).name;
const Field & column_name = *getIdentifierName(command_ast->column);
res.column_name = column_name;
return res;
}

View File

@ -304,16 +304,16 @@ void registerStorageFile(StorageFactory & factory)
{
/// Will use FD if engine_args[1] is int literal or identifier with std* name
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(engine_args[1].get()))
if (auto opt_name = getIdentifierName(engine_args[1]))
{
if (identifier->name == "stdin")
if (*opt_name == "stdin")
source_fd = STDIN_FILENO;
else if (identifier->name == "stdout")
else if (*opt_name == "stdout")
source_fd = STDOUT_FILENO;
else if (identifier->name == "stderr")
else if (*opt_name == "stderr")
source_fd = STDERR_FILENO;
else
throw Exception("Unknown identifier '" + identifier->name + "' in second arg of File storage constructor",
throw Exception("Unknown identifier '" + *opt_name + "' in second arg of File storage constructor",
ErrorCodes::UNKNOWN_IDENTIFIER);
}
else if (const ASTLiteral * literal = typeid_cast<const ASTLiteral *>(engine_args[1].get()))

View File

@ -6,7 +6,6 @@
#include <Storages/StorageHDFS.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <IO/ReadBufferFromHDFS.h>
#include <Formats/FormatFactory.h>
@ -16,7 +15,6 @@
#include <DataStreams/OwningBlockInputStream.h>
#include <Poco/Path.h>
#include <Common/parseRemoteDescription.h>
#include <Common/typeid_cast.h>
namespace DB

View File

@ -3,7 +3,6 @@
#include <Interpreters/Join.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Core/ColumnNumbers.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/NestedUtils.h>
@ -88,11 +87,11 @@ void registerStorageJoin(StorageFactory & factory)
"Storage Join requires at least 3 parameters: Join(ANY|ALL, LEFT|INNER, keys...).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTIdentifier * strictness_id = typeid_cast<const ASTIdentifier *>(engine_args[0].get());
if (!strictness_id)
auto opt_strictness_id = getIdentifierName(engine_args[0]);
if (!opt_strictness_id)
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const String strictness_str = Poco::toLower(strictness_id->name);
const String strictness_str = Poco::toLower(*opt_strictness_id);
ASTTableJoin::Strictness strictness;
if (strictness_str == "any")
strictness = ASTTableJoin::Strictness::Any;
@ -101,11 +100,11 @@ void registerStorageJoin(StorageFactory & factory)
else
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const ASTIdentifier * kind_id = typeid_cast<const ASTIdentifier *>(engine_args[1].get());
if (!kind_id)
auto opt_kind_id = getIdentifierName(engine_args[1]);
if (!opt_kind_id)
throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const String kind_str = Poco::toLower(kind_id->name);
const String kind_str = Poco::toLower(*opt_kind_id);
ASTTableJoin::Kind kind;
if (kind_str == "left")
kind = ASTTableJoin::Kind::Left;
@ -122,11 +121,11 @@ void registerStorageJoin(StorageFactory & factory)
key_names.reserve(engine_args.size() - 2);
for (size_t i = 2, size = engine_args.size(); i < size; ++i)
{
const ASTIdentifier * key = typeid_cast<const ASTIdentifier *>(engine_args[i].get());
if (!key)
auto opt_key = getIdentifierName(engine_args[i]);
if (!opt_key)
throw Exception("Parameter №" + toString(i + 1) + " of storage Join don't look like column name.", ErrorCodes::BAD_ARGUMENTS);
key_names.push_back(key->name);
key_names.push_back(*opt_key);
}
auto & settings = args.context.getSettingsRef();

View File

@ -41,6 +41,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
{ "is_in_sorting_key", std::make_shared<DataTypeUInt8>() },
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
{ "is_in_sampling_key", std::make_shared<DataTypeUInt8>() },
{ "compression_codec", std::make_shared<DataTypeString>() },
}));
}
@ -86,6 +87,7 @@ protected:
NamesAndTypesList columns;
ColumnDefaults column_defaults;
ColumnComments column_comments;
ColumnCodecs column_codecs;
Names cols_required_for_partition_key;
Names cols_required_for_sorting_key;
Names cols_required_for_primary_key;
@ -114,6 +116,7 @@ protected:
}
columns = storage->getColumns().getAll();
column_codecs = storage->getColumns().codecs;
column_defaults = storage->getColumns().defaults;
column_comments = storage->getColumns().comments;
@ -219,6 +222,20 @@ protected:
res_columns[res_index++]->insert(find_in_vector(cols_required_for_sampling));
}
{
const auto it = column_codecs.find(column.name);
if (it == std::end(column_codecs))
{
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
}
else
{
if (columns_mask[src_index++])
res_columns[res_index++]->insert("CODEC(" + it->second->getCodecDesc() + ")");
}
}
++rows_count;
}
}

View File

@ -96,11 +96,9 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns)
if (!isValidFunction(expression->children[i], columns))
return false;
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(&*expression))
{
if (identifier->general())
return columns.count(identifier->name);
}
if (auto opt_name = getColumnIdentifierName(expression))
return columns.count(*opt_name);
return true;
}

View File

@ -76,7 +76,7 @@ static bool isCompatible(const IAST & node)
return true;
}
if (typeid_cast<const ASTIdentifier *>(&node))
if (isIdentifier(&node))
return true;
return false;

View File

@ -65,9 +65,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
}
else
{
if (auto ast_cluster = typeid_cast<const ASTIdentifier *>(args[arg_num].get()))
cluster_name = ast_cluster->name;
else
if (!getIdentifierName(args[arg_num], cluster_name))
cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern");
}
++arg_num;
@ -132,9 +130,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
/// ExpressionAnalyzer will be created in InterpreterSelectQuery that will meet these `Identifier` when processing the request.
/// We need to mark them as the name of the database or table, because the default value is column.
for (auto & arg : args)
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
id->setSpecial();
for (auto ast : args)
setIdentifierSpecial(ast);
ClusterPtr cluster;
if (!cluster_name.empty())

View File

@ -0,0 +1,33 @@
2018-01-01 1
2018-01-01 2
CODEC(ZSTD)
2018-01-01 1 default_value
2018-01-01 2 default_value
2018-01-01 3 3
2018-01-01 4 4
CODEC(NONE)
2018-01-01 1 default_value
2018-01-01 2 default_value
2018-01-01 3 3
2018-01-01 4 4
2018-01-01 5 5
2018-01-01 6 6
2018-01-01 1 default_value
2018-01-01 2 default_value
2018-01-01 3 3
2018-01-01 4 4
2018-01-01 5 5
2018-01-01 6 6
CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
2018-01-01 1 default_value
2018-01-01 2 default_value
2018-01-01 3 3
2018-01-01 4 4
2018-01-01 5 5
2018-01-01 6 6
2018-01-01 7 7
2018-01-01 8 8
CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
CODEC(NONE, LZ4, LZ4HC, ZSTD)
2
1

View File

@ -0,0 +1,88 @@
SET send_logs_level = 'none';
DROP TABLE IF EXISTS test.alter_compression_codec;
CREATE TABLE test.alter_compression_codec (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id;
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 1);
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 2);
SELECT * FROM test.alter_compression_codec ORDER BY id;
ALTER TABLE test.alter_compression_codec ADD COLUMN alter_column String DEFAULT 'default_value' CODEC(ZSTD);
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 3, '3');
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 4, '4');
SELECT * FROM test.alter_compression_codec ORDER BY id;
ALTER TABLE test.alter_compression_codec MODIFY COLUMN alter_column CODEC(NONE);
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 5, '5');
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 6, '6');
SELECT * FROM test.alter_compression_codec ORDER BY id;
OPTIMIZE TABLE test.alter_compression_codec FINAL;
SELECT * FROM test.alter_compression_codec ORDER BY id;
ALTER TABLE test.alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE);
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 7, '7');
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 8, '8');
OPTIMIZE TABLE test.alter_compression_codec FINAL;
SELECT * FROM test.alter_compression_codec ORDER BY id;
ALTER TABLE test.alter_compression_codec MODIFY COLUMN alter_column FixedString(100);
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
DROP TABLE IF EXISTS test.alter_compression_codec;
DROP TABLE IF EXISTS test.alter_bad_codec;
CREATE TABLE test.alter_bad_codec (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = MergeTree() ORDER BY tuple();
ALTER TABLE test.alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(gbdgkjsdh); -- { serverError 432 }
ALTER TABLE test.alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(ZSTD(100)); -- { serverError 433 }
DROP TABLE IF EXISTS test.alter_bad_codec;
DROP TABLE IF EXISTS test.large_alter_table;
DROP TABLE IF EXISTS test.store_of_hash;
CREATE TABLE test.large_alter_table (
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)),
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC),
data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4)
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
INSERT INTO test.large_alter_table SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000;
CREATE TABLE test.store_of_hash (hash UInt64) ENGINE = Memory();
INSERT INTO test.store_of_hash SELECT sum(cityHash64(*)) FROM test.large_alter_table;
ALTER TABLE test.large_alter_table MODIFY COLUMN data CODEC(NONE, LZ4, LZ4HC, ZSTD);
OPTIMIZE TABLE test.large_alter_table;
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'large_alter_table' AND name = 'data';
DETACH TABLE test.large_alter_table;
ATTACH TABLE test.large_alter_table;
INSERT INTO test.store_of_hash SELECT sum(cityHash64(*)) FROM test.large_alter_table;
SELECT COUNT(hash) FROM test.store_of_hash;
SELECT COUNT(DISTINCT hash) FROM test.store_of_hash;
DROP TABLE IF EXISTS test.large_alter_table;
DROP TABLE IF EXISTS test.store_of_hash;

View File

@ -0,0 +1,60 @@
# UUID {#uuid-data-type}
Universally unique identifier (UUID) is a 16-byte number used to identify the records. For detailed information about the UUID, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier).
The example of UUID type value is represented below:
```
61f0c404-5cb3-11e7-907b-a6006ad3dba0
```
If you do not specify the UUID column value when inserting a new record, the UUID value is filled with zero:
```
00000000-0000-0000-0000-000000000000
```
## How to generate
To generate the UUID value, ClickHouse provides the [generateUUIDv4](../query_language/functions/uuid_functions.md) function.
## Usage example
**Example 1**
This example demonstrates creating a table with the UUID type column and inserting a value into the table.
``` sql
:) CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog
:) INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1'
:) SELECT * FROM t_uuid
┌────────────────────────────────────x─┬─y─────────┐
│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │
└──────────────────────────────────────┴───────────┘
```
**Example 2**
In this example, the UUID column value is not specified when inserting a new record.
``` sql
:) INSERT INTO t_uuid (y) VALUES ('Example 2')
:) SELECT * FROM t_uuid
┌────────────────────────────────────x─┬─y─────────┐
│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │
│ 00000000-0000-0000-0000-000000000000 │ Example 2 │
└──────────────────────────────────────┴───────────┘
```
## Restrictions
The UUID data type only supports functions which [String](string.md) data type also supports (for example, [min](../query_language/agg_functions/reference.md#agg_function-min), [max](../query_language/agg_functions/reference.md#agg_function-max), and [count](../query_language/agg_functions/reference.md#agg_function-count)).
The UUID data type is not supported by arithmetic operations (for example, [abs](../query_language/functions/arithmetic_functions.md#arithm_func-abs)) neither aggregate functions, such as [sum](../query_language/agg_functions/reference.md#agg_function-sum) and [avg](../query_language/agg_functions/reference.md#agg_function-avg).
[Original article](https://clickhouse.yandex/docs/en/data_types/uuid/) <!--hide-->

View File

@ -1,7 +1,7 @@
# Function reference
## count()
## count() {#agg_function-count}
Counts the number of rows. Accepts zero arguments and returns UInt64.
The syntax `COUNT(DISTINCT x)` is not supported. The separate `uniq` aggregate function exists for this purpose.
@ -179,15 +179,15 @@ binary decimal
01101000 = 104
```
## min(x)
## min(x) {#agg_function-min}
Calculates the minimum.
## max(x)
## max(x) {#agg_function-max}
Calculates the maximum.
## argMin(arg, val)
## argMin(arg, val) {#agg_function-argMin}
Calculates the 'arg' value for a minimal 'val' value. If there are several different values of 'arg' for minimal values of 'val', the first of these values encountered is output.
@ -206,7 +206,7 @@ SELECT argMin(user, salary) FROM salary
└──────────────────────┘
```
## argMax(arg, val)
## argMax(arg, val) {#agg_function-argMax}
Calculates the 'arg' value for a maximum 'val' value. If there are several different values of 'arg' for maximum values of 'val', the first of these values encountered is output.
@ -259,7 +259,7 @@ GROUP BY timeslot
└─────────────────────┴──────────────────────────────────────────────┘
```
## avg(x)
## avg(x) {#agg_function-avg}
Calculates the average.
Only works for numbers.

View File

@ -59,7 +59,7 @@ An exception is thrown when dividing by zero or when dividing a minimal negative
Calculates a number with the reverse sign. The result is always signed.
## abs(a)
## abs(a) {#arithm_func-abs}
Calculates the absolute value of the number (a). That is, if a &lt; 0, it returns -a. For unsigned types it doesn't do anything. For signed integer types, it returns an unsigned number.

View File

@ -1,6 +1,6 @@
# Functions for working with external dictionaries {#ext_dict_functions}
For information on connecting and configuring external dictionaries, see "[External dictionaries](../dicts/external_dicts.md)".
For information on connecting and configuring external dictionaries, see [External dictionaries](../dicts/external_dicts.md).
## dictGetUInt8, dictGetUInt16, dictGetUInt32, dictGetUInt64
@ -19,7 +19,7 @@ For information on connecting and configuring external dictionaries, see "[Exter
- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key.`dict_name` and `attr_name` are constant strings.`id`must be UInt64.
If there is no `id` key in the dictionary, it returns the default value specified in the dictionary description.
## dictGetTOrDefault
## dictGetTOrDefault {#ext_dict_functions_dictGetTOrDefault}
`dictGetT('dict_name', 'attr_name', id, default)`

View File

@ -0,0 +1,108 @@
# Functions for working with UUID
The functions for working with UUID are listed below.
## generateUUIDv4 {#uuid_function-generate}
Generates [UUID](../../data_types/uuid.md) of [version 4](https://tools.ietf.org/html/rfc4122#section-4.4).
```sql
generateUUIDv4()
```
**Returned value**
The UUID type value.
**Usage example**
This example demonstrates creating a table with the UUID type column and inserting a value into the table.
``` sql
:) CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog
:) INSERT INTO t_uuid SELECT generateUUIDv4()
:) SELECT * FROM t_uuid
┌────────────────────────────────────x─┐
│ f4bf890f-f9dc-4332-ad5c-0c18e73f28e9 │
└──────────────────────────────────────┘
```
## toUUID (x)
Converts String type value to UUID type.
```sql
toUUID(String)
```
**Returned value**
The UUID type value.
**Usage example**
``` sql
:) SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid
┌─────────────────────────────────uuid─┐
│ 61f0c404-5cb3-11e7-907b-a6006ad3dba0 │
└──────────────────────────────────────┘
```
## UUIDStringToNum
Accepts a string containing 36 characters in the format `xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, and returns it as a set of bytes in a [FixedString(16)](../../data_types/fixedstring.md).
``` sql
UUIDStringToNum(String)
```
**Returned value**
FixedString(16)
**Usage examples**
``` sql
:) SELECT
'612f3c40-5d3b-217e-707b-6a546a3d7b29' AS uuid,
UUIDStringToNum(uuid) AS bytes
┌─uuid─────────────────────────────────┬─bytes────────────┐
│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │
└──────────────────────────────────────┴──────────────────┘
```
## UUIDNumToString
Accepts a [FixedString(16)](../../data_types/fixedstring.md) value, and returns a string containing 36 characters in text format.
``` sql
UUIDNumToString(FixedString(16))
```
**Returned value**
String.
**Usage example**
``` sql
SELECT
'a/<@];!~p{jTj={)' AS bytes,
UUIDNumToString(toFixedString(bytes, 16)) AS uuid
┌─bytes────────────┬─uuid─────────────────────────────────┐
│ a/<@];!~p{jTj={) │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │
└──────────────────┴──────────────────────────────────────┘
```
## See also
- [dictGetUUID](ext_dict_functions.md)
- [dictGetUUIDOrDefault](ext_dict_functions#ext_dict_functions_dictGetTOrDefault)
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/uuid_function/) <!--hide-->

View File

@ -26,52 +26,61 @@ SQL-style and C-style comments are supported.
SQL-style comments: from `--` to the end of the line. The space after `--` can be omitted.
Comments in C-style: from `/*` to `*/`. These comments can be multiline. Spaces are not required here, either.
## Keywords
## Keywords {#syntax-keywords}
Keywords (such as `SELECT`) are not case-sensitive. Everything else (column names, functions, and so on), in contrast to standard SQL, is case-sensitive. Keywords are not reserved (they are just parsed as keywords in the corresponding context).
Keywords (such as `SELECT`) are not case-sensitive. Everything else (column names, functions, and so on), in contrast to standard SQL, is case-sensitive.
## Identifiers
Keywords are not reserved (they are just parsed as keywords in the corresponding context). If you use [identifiers](#syntax-identifiers) the same as the keywords, enclose them into quotes. For example, the query `SELECT "FROM" FROM table_name` is valid if the table `table_name` has column with the name `"FROM"`.
Identifiers (column names, functions, and data types) can be quoted or non-quoted.
Non-quoted identifiers start with a Latin letter or underscore, and continue with a Latin letter, underscore, or number. In other words, they must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$`. Examples: `x, _1, X_y__Z123_.`
## Identifiers {#syntax-identifiers}
Quoted identifiers are placed in reversed quotation marks `` `id` `` (the same as in MySQL), and can indicate any set of bytes (non-empty). In addition, symbols (for example, the reverse quotation mark) inside this type of identifier can be backslash-escaped. Escaping rules are the same as for string literals (see below).
We recommend using identifiers that do not need to be quoted.
Identifiers are:
- Cluster, database, table, partition and column names;
- Functions;
- Data types;
- [Expression aliases](#syntax-expression_aliases).
Identifiers can be quoted or non-quoted. It is recommended to use non-quoted identifiers.
Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#syntax-keywords). Examples: `x, _1, X_y__Z123_.`
If you want to use identifiers the same as keywords or you want to use other symbols in identifiers, quote it using double quotes or backticks, for example, `"id"`, `` `id` ``.
## Literals
There are numeric literals, string literals, and compound literals.
There are: numeric, string, compound and `NULL` literals.
### Numeric Literals
### Numeric
A numeric literal tries to be parsed:
- First as a 64-bit signed number, using the 'strtoull' function.
- If unsuccessful, as a 64-bit unsigned number, using the 'strtoll' function.
- If unsuccessful, as a floating-point number using the 'strtod' function.
- First as a 64-bit signed number, using the [strtoull](https://en.cppreference.com/w/cpp/string/byte/strtoul) function.
- If unsuccessful, as a 64-bit unsigned number, using the [strtoll](https://en.cppreference.com/w/cpp/string/byte/strtol) function.
- If unsuccessful, as a floating-point number using the [strtod](https://en.cppreference.com/w/cpp/string/byte/strtof) function.
- Otherwise, an error is returned.
The corresponding value will have the smallest type that the value fits in.
For example, 1 is parsed as UInt8, but 256 is parsed as UInt16. For more information, see "Data types".
For example, 1 is parsed as `UInt8`, but 256 is parsed as `UInt16`. For more information, see [Data types](../data_types/index.md).
Examples: `1`, `18446744073709551615`, `0xDEADBEEF`, `01`, `0.1`, `1e100`, `-1e-100`, `inf`, `nan`.
### String Literals
### String
Only string literals in single quotes are supported. The enclosed characters can be backslash-escaped. The following escape sequences have a corresponding special value: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. In all other cases, escape sequences in the format `\c`, where "c" is any character, are converted to "c". This means that you can use the sequences `\'`and`\\`. The value will have the String type.
Only string literals in single quotes are supported. The enclosed characters can be backslash-escaped. The following escape sequences have a corresponding special value: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. In all other cases, escape sequences in the format `\c`, where `c` is any character, are converted to `c`. This means that you can use the sequences `\'`and`\\`. The value will have the String type.
The minimum set of characters that you need to escape in string literals: `'` and `\`.
The minimum set of characters that you need to escape in string literals: `'` and `\`. Single quote can be escaped with the single quote, literals `'It\'s'` and `'It''s'` are equal.
### Compound Literals
### Compound
Constructions are supported for arrays: `[1, 2, 3]` and tuples: `(1, 'Hello, world!', 2)`..
Actually, these are not literals, but expressions with the array creation operator and the tuple creation operator, respectively.
For more information, see the section "Operators2".
An array must consist of at least one item, and a tuple must have at least two items.
Tuples have a special purpose for use in the IN clause of a SELECT query. Tuples can be obtained as the result of a query, but they can't be saved to a database (with the exception of Memory-type tables).
Tuples have a special purpose for use in the `IN` clause of a `SELECT` query. Tuples can be obtained as the result of a query, but they can't be saved to a database (with the exception of [Memory](../operations/table_engines/memory.md) tables).
### NULL Literal {#null-literal}
### NULL {#null-literal}
Indicates that the value is missing.
@ -92,13 +101,13 @@ There are regular and aggregate functions (see the section "Aggregate functions"
Operators are converted to their corresponding functions during query parsing, taking their priority and associativity into account.
For example, the expression `1 + 2 * 3 + 4` is transformed to `plus(plus(1, multiply(2, 3)), 4)`.
For more information, see the section "Operators" below.
## Data Types and Database Table Engines
Data types and table engines in the `CREATE` query are written the same way as identifiers or functions. In other words, they may or may not contain an arguments list in brackets. For more information, see the sections "Data types," "Table engines," and "CREATE".
## Expression Aliases
## Expression Aliases {#syntax-expression_aliases}
Alias is a user defined name for an expression in a query.
@ -106,27 +115,27 @@ Alias is a user defined name for an expression in a query.
expr AS alias
```
- `AS`keyword for defining aliases. You can define alias for a table name or a column name in SELECT clause skipping `AS` keyword.
- `AS`Keyword for defining aliases. You can define alias for a table name or a column name in the `SELECT` clause skipping `AS` keyword.
For example, `SELECT b.column_name from t b`.
For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`.
In the [CAST function](functions/type_conversion_functions.md), the `AS` keyword has another meaning. See the description of the function.
- `expr`any expression supported by ClickHouse.
- `expr`Any expression supported by ClickHouse.
For example `SELECT column_name * 2 AS double FROM some_table`.
- `alias`[string literal](#string-literals). If an alias contains spaces, enclose it in double quotes or backticks.
- `alias`Name for `expr`. Aliases should comply with the [identifiers](#syntax-identifiers) syntax.
For example, `SELECT "table t".col_name FROM t AS "table t"`.
For example, `SELECT "table t".column_name FROM table_name AS "table t"`.
### Peculiarities of Use
Aliases are global for a query or subquery and you can define alias in any part of a query for any expression. For example, `SELECT (1 AS n) + 2, n`.
Aliases are not visible in between subqueries. For example, while executing the query `SELECT (SELECT sum(b.a) + num FROM b) - a.a AS num FROM a` ClickHouse generates exception `Unknown identifier: num`.
Aliases are not visible in subqueries. For example, while executing the query `SELECT (SELECT sum(b.a) + num FROM b) - a.a AS num FROM a` ClickHouse generates the exception `Unknown identifier: num`.
If alias is defined for result columns in SELECT clause in a subquery, these columns are visible in outer query. For example, `SELECT n + m FROM (SELECT 1 AS n, 2 AS m)`.
If an alias is defined for result columns in `SELECT` clause in a subquery, these columns are visible in outer query. For example, `SELECT n + m FROM (SELECT 1 AS n, 2 AS m)`.
Be careful with aliases the same as column or table names. Let's consider the following example:

View File

@ -157,7 +157,7 @@ x=1 y=\N
clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
```
&ast;По умолчанию — `,`. См. настройку [format_csv_delimiter](/operations/settings/settings/#settings-format_csv_delimiter) для дополнительной информации.
&ast;По умолчанию — `,`. См. настройку [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) для дополнительной информации.
При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты.

View File

@ -245,7 +245,7 @@ TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster]
Удаляет все данные из таблицы. Если условие `IF EXISTS` не указано, запрос вернет ошибку, если таблицы не существует.
Запрос `TRUNCATE` не поддерживается для следующих движков: [View](../operations/table_engines/view.md), [File](../operations/table_engines/file.md), [URL](../operations/table_engines/url.md) and [Null](../operations/table_engines/null.md).
Запрос `TRUNCATE` не поддерживается для следующих движков: [View](../operations/table_engines/view.md), [File](../operations/table_engines/file.md), [URL](../operations/table_engines/url.md) и [Null](../operations/table_engines/null.md).
## USE

View File

@ -39,6 +39,7 @@ nav:
- 'Boolean': 'data_types/boolean.md'
- 'String': 'data_types/string.md'
- 'FixedString(N)': 'data_types/fixedstring.md'
- 'UUID': 'data_types/uuid.md'
- 'Date': 'data_types/date.md'
- 'DateTime': 'data_types/datetime.md'
- 'Enum': 'data_types/enum.md'
@ -81,6 +82,7 @@ nav:
- 'Hash': 'query_language/functions/hash_functions.md'
- 'Generating Pseudo-Random Numbers': 'query_language/functions/random_functions.md'
- 'Encoding': 'query_language/functions/encoding_functions.md'
- 'Working with UUID': 'query_language/functions/uuid_functions.md'
- 'Working with URLs': 'query_language/functions/url_functions.md'
- 'Working with IP Addresses': 'query_language/functions/ip_address_functions.md'
- 'Working with JSON.': 'query_language/functions/json_functions.md'