Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2019-01-15 21:56:50 +03:00
commit cd77d43015
115 changed files with 1846 additions and 1076 deletions

3
.gitignore vendored
View File

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

3
.gitmodules vendored
View File

@ -61,3 +61,6 @@
[submodule "contrib/libgsasl"] [submodule "contrib/libgsasl"]
path = contrib/libgsasl path = contrib/libgsasl
url = https://github.com/ClickHouse-Extras/libgsasl.git 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) if (USE_STATIC_LIBRARIES AND NOT OS_FREEBSD)
find_library (SASL2_LIBRARY sasl2) find_library (SASL2_LIBRARY sasl2)
endif () endif ()
set (CPPKAFKA_LIBRARY cppkafka) # TODO: try to use unbundled version.
endif () endif ()
if (RDKAFKA_LIB AND RDKAFKA_INCLUDE_DIR) if (RDKAFKA_LIB AND RDKAFKA_INCLUDE_DIR)
set (USE_RDKAFKA 1) set (USE_RDKAFKA 1)
set (RDKAFKA_LIBRARY ${RDKAFKA_LIB} ${OPENSSL_LIBRARIES}) set (RDKAFKA_LIBRARY ${RDKAFKA_LIB} ${OPENSSL_LIBRARIES})
set (CPPKAFKA_LIBRARY cppkafka)
if (SASL2_LIBRARY) if (SASL2_LIBRARY)
list (APPEND RDKAFKA_LIBRARY ${SASL2_LIBRARY}) list (APPEND RDKAFKA_LIBRARY ${SASL2_LIBRARY})
endif () endif ()
@ -35,9 +37,10 @@ elseif (NOT MISSING_INTERNAL_RDKAFKA_LIBRARY AND NOT ARCH_ARM)
set (USE_INTERNAL_RDKAFKA_LIBRARY 1) set (USE_INTERNAL_RDKAFKA_LIBRARY 1)
set (RDKAFKA_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src") set (RDKAFKA_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src")
set (RDKAFKA_LIBRARY rdkafka) set (RDKAFKA_LIBRARY rdkafka)
set (CPPKAFKA_LIBRARY cppkafka)
set (USE_RDKAFKA 1) set (USE_RDKAFKA 1)
endif () endif ()
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}) target_include_directories(rdkafka BEFORE PRIVATE ${OPENSSL_INCLUDE_DIR})
endif () endif ()
if (USE_RDKAFKA)
add_subdirectory (cppkafka-cmake)
endif()
if (ENABLE_ODBC AND USE_INTERNAL_ODBC_LIBRARY) if (ENABLE_ODBC AND USE_INTERNAL_ODBC_LIBRARY)
add_subdirectory (unixodbc-cmake) add_subdirectory (unixodbc-cmake)
endif () 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(RDKAFKA_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/librdkafka/src)
set(SRCS set(SRCS
${RDKAFKA_SOURCE_DIR}/crc32c.c ${RDKAFKA_SOURCE_DIR}/crc32c.c
${RDKAFKA_SOURCE_DIR}/rdaddr.c ${RDKAFKA_SOURCE_DIR}/rdaddr.c
${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdavl.c
${RDKAFKA_SOURCE_DIR}/rdbuf.c ${RDKAFKA_SOURCE_DIR}/rdbuf.c
${RDKAFKA_SOURCE_DIR}/rdcrc32.c ${RDKAFKA_SOURCE_DIR}/rdcrc32.c
${RDKAFKA_SOURCE_DIR}/rdkafka.c ${RDKAFKA_SOURCE_DIR}/rdkafka.c
${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_broker.c ${RDKAFKA_SOURCE_DIR}/rdkafka_background.c
${RDKAFKA_SOURCE_DIR}/rdkafka_buf.c ${RDKAFKA_SOURCE_DIR}/rdkafka_broker.c
${RDKAFKA_SOURCE_DIR}/rdkafka_cgrp.c ${RDKAFKA_SOURCE_DIR}/rdkafka_buf.c
${RDKAFKA_SOURCE_DIR}/rdkafka_conf.c ${RDKAFKA_SOURCE_DIR}/rdkafka_cgrp.c
${RDKAFKA_SOURCE_DIR}/rdkafka_event.c ${RDKAFKA_SOURCE_DIR}/rdkafka_conf.c
${RDKAFKA_SOURCE_DIR}/rdkafka_feature.c ${RDKAFKA_SOURCE_DIR}/rdkafka_event.c
${RDKAFKA_SOURCE_DIR}/rdkafka_lz4.c ${RDKAFKA_SOURCE_DIR}/rdkafka_feature.c
${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c ${RDKAFKA_SOURCE_DIR}/rdkafka_idempotence.c
${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c ${RDKAFKA_SOURCE_DIR}/rdkafka_lz4.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c
${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_writer.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c
${RDKAFKA_SOURCE_DIR}/rdkafka_offset.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c
${RDKAFKA_SOURCE_DIR}/rdkafka_op.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_writer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c ${RDKAFKA_SOURCE_DIR}/rdkafka_offset.c
${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c ${RDKAFKA_SOURCE_DIR}/rdkafka_op.c
${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c ${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c
${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c
${RDKAFKA_SOURCE_DIR}/rdkafka_request.c ${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c
${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c ${RDKAFKA_SOURCE_DIR}/rdkafka_request.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c ${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_subscription.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c
${RDKAFKA_SOURCE_DIR}/rdkafka_timer.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c
${RDKAFKA_SOURCE_DIR}/rdkafka_topic.c ${RDKAFKA_SOURCE_DIR}/rdkafka_subscription.c
${RDKAFKA_SOURCE_DIR}/rdkafka_transport.c ${RDKAFKA_SOURCE_DIR}/rdkafka_timer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_interceptor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_topic.c
${RDKAFKA_SOURCE_DIR}/rdkafka_header.c ${RDKAFKA_SOURCE_DIR}/rdkafka_transport.c
${RDKAFKA_SOURCE_DIR}/rdlist.c ${RDKAFKA_SOURCE_DIR}/rdkafka_interceptor.c
${RDKAFKA_SOURCE_DIR}/rdlog.c ${RDKAFKA_SOURCE_DIR}/rdkafka_header.c
${RDKAFKA_SOURCE_DIR}/rdmurmur2.c ${RDKAFKA_SOURCE_DIR}/rdlist.c
${RDKAFKA_SOURCE_DIR}/rdports.c ${RDKAFKA_SOURCE_DIR}/rdlog.c
${RDKAFKA_SOURCE_DIR}/rdrand.c ${RDKAFKA_SOURCE_DIR}/rdmurmur2.c
${RDKAFKA_SOURCE_DIR}/rdregex.c ${RDKAFKA_SOURCE_DIR}/rdports.c
${RDKAFKA_SOURCE_DIR}/rdstring.c ${RDKAFKA_SOURCE_DIR}/rdrand.c
${RDKAFKA_SOURCE_DIR}/rdunittest.c ${RDKAFKA_SOURCE_DIR}/rdregex.c
${RDKAFKA_SOURCE_DIR}/rdvarint.c ${RDKAFKA_SOURCE_DIR}/rdstring.c
${RDKAFKA_SOURCE_DIR}/snappy.c ${RDKAFKA_SOURCE_DIR}/rdunittest.c
${RDKAFKA_SOURCE_DIR}/tinycthread.c ${RDKAFKA_SOURCE_DIR}/rdvarint.c
${RDKAFKA_SOURCE_DIR}/xxhash.c ${RDKAFKA_SOURCE_DIR}/snappy.c
${RDKAFKA_SOURCE_DIR}/lz4.c ${RDKAFKA_SOURCE_DIR}/tinycthread.c
${RDKAFKA_SOURCE_DIR}/lz4frame.c ${RDKAFKA_SOURCE_DIR}/tinycthread_extra.c
${RDKAFKA_SOURCE_DIR}/lz4hc.c ${RDKAFKA_SOURCE_DIR}/xxhash.c
${RDKAFKA_SOURCE_DIR}/rdgz.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}) 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_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR})
target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY})

View File

@ -71,4 +71,8 @@
#define HAVE_PTHREAD_SETNAME_GNU 1 #define HAVE_PTHREAD_SETNAME_GNU 1
// python // python
//#define HAVE_PYTHON 1 //#define HAVE_PYTHON 1
// C11 threads
#if (__STDC_VERSION__ >= 201112L) && !defined(__STDC_NO_THREADS__)
# define WITH_C11THREADS 1
#endif
#endif /* _CONFIG_H_ */ #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) if (USE_RDKAFKA)
target_link_libraries (dbms PRIVATE ${RDKAFKA_LIBRARY}) target_link_libraries (dbms PRIVATE ${RDKAFKA_LIBRARY})
target_link_libraries (dbms PRIVATE ${CPPKAFKA_LIBRARY})
if (NOT USE_INTERNAL_RDKAFKA_LIBRARY) if (NOT USE_INTERNAL_RDKAFKA_LIBRARY)
target_include_directories (dbms SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR}) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR})
endif () endif ()

View File

@ -1,5 +1,5 @@
add_library (clickhouse-client-lib ${LINK_MODE} Client.cpp) add_library (clickhouse-client-lib ${LINK_MODE} Client.cpp)
target_link_libraries (clickhouse-client-lib PRIVATE clickhouse_common_io clickhouse_functions clickhouse_aggregate_functions ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_link_libraries (clickhouse-client-lib PRIVATE clickhouse_common_config clickhouse_functions clickhouse_aggregate_functions clickhouse_common_io ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY})
if (READLINE_INCLUDE_DIR) if (READLINE_INCLUDE_DIR)
target_include_directories (clickhouse-client-lib SYSTEM PRIVATE ${READLINE_INCLUDE_DIR}) target_include_directories (clickhouse-client-lib SYSTEM PRIVATE ${READLINE_INCLUDE_DIR})
endif () endif ()

View File

@ -9,7 +9,7 @@ add_library (clickhouse-odbc-bridge-lib ${LINK_MODE}
validateODBCConnectionString.cpp validateODBCConnectionString.cpp
) )
target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE clickhouse_common_io daemon dbms) target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE clickhouse_dictionaries daemon dbms clickhouse_common_io)
target_include_directories (clickhouse-odbc-bridge-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include) target_include_directories (clickhouse-odbc-bridge-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include)
if (USE_POCO_SQLODBC) if (USE_POCO_SQLODBC)

View File

@ -411,7 +411,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->setMarkCache(mark_cache_size); global_context->setMarkCache(mark_cache_size);
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", std::numeric_limits<UInt64>::max()); size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", 500);
if (compiled_expression_cache_size) if (compiled_expression_cache_size)
global_context->setCompiledExpressionCache(compiled_expression_cache_size); global_context->setCompiledExpressionCache(compiled_expression_cache_size);
#endif #endif

View File

@ -1,4 +1,4 @@
#include <TableFunctions/parseRemoteDescription.h> #include "parseRemoteDescription.h"
#include <Common/Exception.h> #include <Common/Exception.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>

View File

@ -0,0 +1,151 @@
#include <Compression/CompressionCodecDelta.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <common/unaligned.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <IO/WriteHelpers.h>
#include <cstdlib>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
}
CompressionCodecDelta::CompressionCodecDelta(UInt8 delta_bytes_size_)
: delta_bytes_size(delta_bytes_size_)
{
}
UInt8 CompressionCodecDelta::getMethodByte() const
{
return static_cast<UInt8>(CompressionMethodByte::Delta);
}
String CompressionCodecDelta::getCodecDesc() const
{
return "Delta(" + toString(delta_bytes_size) + ")";
}
namespace
{
template <typename T>
void compressDataForType(const char * source, UInt32 source_size, char * dest)
{
if (source_size % sizeof(T) != 0)
throw Exception("Cannot delta compress, data size " + toString(source_size) + " is not aligned to " + toString(sizeof(T)), ErrorCodes::CANNOT_COMPRESS);
const auto * source_with_type = reinterpret_cast<const T *>(source);
auto * dest_with_type = reinterpret_cast<T *>(dest);
if (source_size > 0)
dest_with_type[0] = source_with_type[0];
for (size_t dest_index = 1, dest_end = source_size / sizeof(T); dest_index < dest_end; ++dest_index)
unalignedStore<T>(&dest_with_type[dest_index], source_with_type[dest_index] - source_with_type[dest_index - 1]);
}
template <typename T>
void decompressDataForType(const char * source, UInt32 source_size, char * dest)
{
if (source_size % sizeof(T) != 0)
throw Exception("Cannot delta decompress, data size " + toString(source_size) + " is not aligned to " + toString(sizeof(T)), ErrorCodes::CANNOT_DECOMPRESS);
const auto * source_with_type = reinterpret_cast<const T *>(source);
auto * dest_with_type = reinterpret_cast<T *>(dest);
if (source_size > 0)
dest_with_type[0] = source_with_type[0];
for (size_t dest_index = 1, dest_end = source_size / sizeof(T); dest_index < dest_end; ++dest_index)
unalignedStore<T>(&dest_with_type[dest_index], source_with_type[dest_index] + dest_with_type[dest_index - 1]);
}
}
UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_size, char * dest) const
{
UInt8 bytes_to_skip = source_size % delta_bytes_size;
dest[0] = delta_bytes_size;
dest[1] = bytes_to_skip;
memcpy(&dest[2], source, bytes_to_skip);
size_t start_pos = 2 + bytes_to_skip;
switch (delta_bytes_size)
{
case 1:
compressDataForType<UInt8>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 2:
compressDataForType<UInt16>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 4:
compressDataForType<UInt32>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
case 8:
compressDataForType<UInt64>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
break;
}
return 1 + 1 + source_size;
}
void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const
{
UInt8 bytes_size = source[0];
UInt8 bytes_to_skip = source[1];
memcpy(dest, &source[2], bytes_to_skip);
UInt32 source_size_no_header = source_size - bytes_to_skip - 2;
switch (bytes_size)
{
case 1:
decompressDataForType<UInt8>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
break;
case 2:
decompressDataForType<UInt16>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
break;
case 4:
decompressDataForType<UInt32>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
break;
case 8:
decompressDataForType<UInt64>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
break;
}
}
void registerCodecDelta(CompressionCodecFactory & factory)
{
UInt8 method_code = UInt8(CompressionMethodByte::Delta);
factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr
{
UInt8 delta_bytes_size = 1;
if (column_type && column_type->haveMaximumSizeOfValue())
{
size_t max_size = column_type->getSizeOfValueInMemory();
if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8)
delta_bytes_size = static_cast<UInt8>(max_size);
}
if (arguments && !arguments->children.empty())
{
if (arguments->children.size() > 1)
throw Exception("Delta codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
const auto children = arguments->children;
const ASTLiteral * literal = static_cast<const ASTLiteral *>(children[0].get());
size_t user_bytes_size = literal->value.safeGet<UInt64>();
if (user_bytes_size != 1 && user_bytes_size != 2 && user_bytes_size != 4 && user_bytes_size != 8)
throw Exception("Delta value for delta codec can be 1, 2, 4 or 8, given " + toString(user_bytes_size), ErrorCodes::ILLEGAL_CODEC_PARAMETER);
delta_bytes_size = static_cast<UInt8>(user_bytes_size);
}
return std::make_shared<CompressionCodecDelta>(delta_bytes_size);
});
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Compression/ICompressionCodec.h>
namespace DB
{
class CompressionCodecDelta : public ICompressionCodec
{
public:
CompressionCodecDelta(UInt8 delta_bytes_size_);
UInt8 getMethodByte() const override;
String getCodecDesc() const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
private:
const UInt8 delta_bytes_size;
};
}

View File

@ -8,6 +8,7 @@
#include "CompressionCodecLZ4.h" #include "CompressionCodecLZ4.h"
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <IO/WriteHelpers.h>
#ifdef __clang__ #ifdef __clang__
#pragma clang diagnostic ignored "-Wold-style-cast" #pragma clang diagnostic ignored "-Wold-style-cast"
@ -35,7 +36,7 @@ String CompressionCodecLZ4::getCodecDesc() const
return "LZ4"; return "LZ4";
} }
UInt32 CompressionCodecLZ4::getCompressedDataSize(UInt32 uncompressed_size) const UInt32 CompressionCodecLZ4::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{ {
return LZ4_COMPRESSBOUND(uncompressed_size); return LZ4_COMPRESSBOUND(uncompressed_size);
} }
@ -61,7 +62,7 @@ void registerCodecLZ4(CompressionCodecFactory & factory)
String CompressionCodecLZ4HC::getCodecDesc() const String CompressionCodecLZ4HC::getCodecDesc() const
{ {
return "LZ4HC"; return "LZ4HC(" + toString(level) + ")";
} }
UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_size, char * dest) const UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_size, char * dest) const

View File

@ -24,7 +24,7 @@ protected:
private: private:
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
mutable LZ4::PerformanceStatistics lz4_stat; mutable LZ4::PerformanceStatistics lz4_stat;
}; };
@ -41,7 +41,7 @@ protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
private: private:
int level; const int level;
}; };
} }

View File

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

View File

@ -15,7 +15,7 @@ public:
String getCodecDesc() const override; String getCodecDesc() const override;
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
protected: protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;

View File

@ -28,10 +28,10 @@ UInt8 CompressionCodecZSTD::getMethodByte() const
String CompressionCodecZSTD::getCodecDesc() const String CompressionCodecZSTD::getCodecDesc() const
{ {
return "ZSTD"; return "ZSTD(" + toString(level) + ")";
} }
UInt32 CompressionCodecZSTD::getCompressedDataSize(UInt32 uncompressed_size) const UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{ {
return ZSTD_compressBound(uncompressed_size); return ZSTD_compressBound(uncompressed_size);
} }

View File

@ -19,7 +19,7 @@ public:
String getCodecDesc() const override; String getCodecDesc() const override;
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
protected: protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
@ -27,7 +27,7 @@ protected:
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
private: private:
int level; const int level;
}; };
} }

View File

@ -10,7 +10,6 @@
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <Compression/CompressionCodecMultiple.h> #include <Compression/CompressionCodecMultiple.h>
#include <Compression/CompressionCodecLZ4.h> #include <Compression/CompressionCodecLZ4.h>
#include <Compression/CompressionCodecNone.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -56,7 +55,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const std::vector<CodecNameWith
return std::make_shared<CompressionCodecMultiple>(result); return std::make_shared<CompressionCodecMultiple>(result);
} }
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type) const
{ {
if (const auto * func = typeid_cast<const ASTFunction *>(ast.get())) if (const auto * func = typeid_cast<const ASTFunction *>(ast.get()))
{ {
@ -65,9 +64,9 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const
for (const auto & inner_codec_ast : func->arguments->children) for (const auto & inner_codec_ast : func->arguments->children)
{ {
if (const auto * family_name = typeid_cast<const ASTIdentifier *>(inner_codec_ast.get())) if (const auto * family_name = typeid_cast<const ASTIdentifier *>(inner_codec_ast.get()))
codecs.emplace_back(getImpl(family_name->name, {})); codecs.emplace_back(getImpl(family_name->name, {}, column_type));
else if (const auto * ast_func = typeid_cast<const ASTFunction *>(inner_codec_ast.get())) else if (const auto * ast_func = typeid_cast<const ASTFunction *>(inner_codec_ast.get()))
codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments)); codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments, column_type));
else else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
} }
@ -88,10 +87,11 @@ CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const
if (family_code_and_creator == family_code_with_codec.end()) if (family_code_and_creator == family_code_with_codec.end())
throw Exception("Unknown codec family code : " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC); throw Exception("Unknown codec family code : " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC);
return family_code_and_creator->second({}); return family_code_and_creator->second({}, nullptr);
} }
CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments) const
CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const
{ {
if (family_name == "Multiple") if (family_name == "Multiple")
throw Exception("Codec MULTIPLE cannot be specified directly", ErrorCodes::UNKNOWN_CODEC); throw Exception("Codec MULTIPLE cannot be specified directly", ErrorCodes::UNKNOWN_CODEC);
@ -101,10 +101,13 @@ CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name,
if (family_and_creator == family_name_with_codec.end()) if (family_and_creator == family_name_with_codec.end())
throw Exception("Unknown codec family: " + family_name, ErrorCodes::UNKNOWN_CODEC); throw Exception("Unknown codec family: " + family_name, ErrorCodes::UNKNOWN_CODEC);
return family_and_creator->second(arguments); return family_and_creator->second(arguments, column_type);
} }
void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator) void CompressionCodecFactory::registerCompressionCodecWithType(
const String & family_name,
std::optional<UInt8> byte_code,
CreatorWithType creator)
{ {
if (creator == nullptr) if (creator == nullptr)
throw Exception("CompressionCodecFactory: the codec family " + family_name + " has been provided a null constructor", throw Exception("CompressionCodecFactory: the codec family " + family_name + " has been provided a null constructor",
@ -118,8 +121,18 @@ void CompressionCodecFactory::registerCompressionCodec(const String & family_nam
throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
} }
void CompressionCodecFactory::registerSimpleCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator)
std::function<CompressionCodecPtr()> creator) {
registerCompressionCodecWithType(family_name, byte_code, [family_name, creator](const ASTPtr & ast, DataTypePtr /* data_type */)
{
return creator(ast);
});
}
void CompressionCodecFactory::registerSimpleCompressionCodec(
const String & family_name,
std::optional<UInt8> byte_code,
SimpleCreator creator)
{ {
registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast) registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast)
{ {
@ -135,7 +148,7 @@ void registerCodecNone(CompressionCodecFactory & factory);
void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory);
void registerCodecMultiple(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory);
void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory);
//void registerCodecDelta(CompressionCodecFactory & factory); void registerCodecDelta(CompressionCodecFactory & factory);
CompressionCodecFactory::CompressionCodecFactory() CompressionCodecFactory::CompressionCodecFactory()
{ {
@ -145,7 +158,7 @@ CompressionCodecFactory::CompressionCodecFactory()
registerCodecZSTD(*this); registerCodecZSTD(*this);
registerCodecMultiple(*this); registerCodecMultiple(*this);
registerCodecLZ4HC(*this); registerCodecLZ4HC(*this);
// registerCodecDelta(*this); registerCodecDelta(*this);
} }
} }

View File

@ -5,6 +5,7 @@
#include <optional> #include <optional>
#include <unordered_map> #include <unordered_map>
#include <ext/singleton.h> #include <ext/singleton.h>
#include <DataTypes/IDataType.h>
#include <Common/IFactoryWithAliases.h> #include <Common/IFactoryWithAliases.h>
#include <Compression/ICompressionCodec.h> #include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h> #include <Compression/CompressionInfo.h>
@ -28,16 +29,19 @@ class CompressionCodecFactory final : public ext::singleton<CompressionCodecFact
{ {
protected: protected:
using Creator = std::function<CompressionCodecPtr(const ASTPtr & parameters)>; using Creator = std::function<CompressionCodecPtr(const ASTPtr & parameters)>;
using CreatorWithType = std::function<CompressionCodecPtr(const ASTPtr & parameters, DataTypePtr column_type)>;
using SimpleCreator = std::function<CompressionCodecPtr()>; using SimpleCreator = std::function<CompressionCodecPtr()>;
using CompressionCodecsDictionary = std::unordered_map<String, Creator>; using CompressionCodecsDictionary = std::unordered_map<String, CreatorWithType>;
using CompressionCodecsCodeDictionary = std::unordered_map<UInt8, Creator>; using CompressionCodecsCodeDictionary = std::unordered_map<UInt8, CreatorWithType>;
public: public:
/// Return default codec (currently LZ4) /// Return default codec (currently LZ4)
CompressionCodecPtr getDefaultCodec() const; CompressionCodecPtr getDefaultCodec() const;
/// Get codec by AST /// Get codec by AST and possible column_type
CompressionCodecPtr get(const ASTPtr & ast) const; /// some codecs can use information about type to improve inner settings
/// but every codec should be able to work without information about type
CompressionCodecPtr get(const ASTPtr & ast, DataTypePtr column_type=nullptr) const;
/// Get codec by method byte (no params available) /// Get codec by method byte (no params available)
CompressionCodecPtr get(const UInt8 byte_code) const; CompressionCodecPtr get(const UInt8 byte_code) const;
@ -46,6 +50,9 @@ public:
CompressionCodecPtr get(const String & family_name, std::optional<int> level) const; CompressionCodecPtr get(const String & family_name, std::optional<int> level) const;
CompressionCodecPtr get(const std::vector<CodecNameWithLevel> & codecs) const; CompressionCodecPtr get(const std::vector<CodecNameWithLevel> & codecs) const;
/// Register codec with parameters and column type
void registerCompressionCodecWithType(const String & family_name, std::optional<UInt8> byte_code, CreatorWithType creator);
/// Register codec with parameters /// Register codec with parameters
void registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator); void registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator);
@ -53,7 +60,7 @@ public:
void registerSimpleCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, SimpleCreator creator); void registerSimpleCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, SimpleCreator creator);
protected: protected:
CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments) const; CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const;
private: private:
CompressionCodecsDictionary family_name_with_codec; CompressionCodecsDictionary family_name_with_codec;

View File

@ -39,6 +39,7 @@ enum class CompressionMethodByte : uint8_t
LZ4 = 0x82, LZ4 = 0x82,
ZSTD = 0x90, ZSTD = 0x90,
Multiple = 0x91, Multiple = 0x91,
Delta = 0x92,
}; };
} }

View File

@ -41,7 +41,7 @@ public:
virtual UInt32 decompress(char * source, UInt32 source_size, char * dest) const; virtual UInt32 decompress(char * source, UInt32 source_size, char * dest) const;
/// Number of bytes, that will be used to compress uncompressed_size bytes with current codec /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec
virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getCompressedDataSize(uncompressed_size); } virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getMaxCompressedDataSize(uncompressed_size); }
/// Some codecs (LZ4, for example) require additional bytes at end of buffer /// Some codecs (LZ4, for example) require additional bytes at end of buffer
virtual UInt32 getAdditionalSizeAtTheEndOfBuffer() const { return 0; } virtual UInt32 getAdditionalSizeAtTheEndOfBuffer() const { return 0; }
@ -61,7 +61,7 @@ public:
protected: protected:
/// Return size of compressed data without header /// Return size of compressed data without header
virtual UInt32 getCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; } virtual UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; }
/// Actually compress data, without header /// Actually compress data, without header
virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0; virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0;

View File

@ -317,9 +317,9 @@ static DataTypePtr create(const ASTPtr & arguments)
params_row[i] = lit->value; 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())) else if (typeid_cast<ASTLiteral *>(arguments->children[0].get()))
{ {

View File

@ -713,7 +713,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
readIntBinary(low_cardinality_state->num_pending_rows, *indexes_stream); readIntBinary(low_cardinality_state->num_pending_rows, *indexes_stream);
} }
size_t num_rows_to_read = std::min(limit, low_cardinality_state->num_pending_rows); size_t num_rows_to_read = std::min<UInt64>(limit, low_cardinality_state->num_pending_rows);
readIndexes(num_rows_to_read); readIndexes(num_rows_to_read);
limit -= num_rows_to_read; limit -= num_rows_to_read;
low_cardinality_state->num_pending_rows -= num_rows_to_read; low_cardinality_state->num_pending_rows -= num_rows_to_read;

View File

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

View File

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

View File

@ -20,9 +20,7 @@ endif()
if(USE_POCO_SQLODBC) if(USE_POCO_SQLODBC)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY}) target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY})
if (NOT USE_INTERNAL_POCO_LIBRARY) target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} ${Poco_SQL_INCLUDE_DIR})
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} ${Poco_SQL_INCLUDE_DIR})
endif()
endif() endif()
if(Poco_Data_FOUND) if(Poco_Data_FOUND)
@ -31,9 +29,7 @@ endif()
if(USE_POCO_DATAODBC) if(USE_POCO_DATAODBC)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_DataODBC_LIBRARY} ${Poco_Data_LIBRARY}) target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_DataODBC_LIBRARY} ${Poco_Data_LIBRARY})
if (NOT USE_INTERNAL_POCO_LIBRARY) target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR})
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR})
endif()
endif() endif()
if(USE_POCO_MONGODB) if(USE_POCO_MONGODB)

View File

@ -61,7 +61,7 @@ void PrettyBlockOutputStream::calculateWidths(
elem.type->serializeText(*elem.column, j, out, format_settings); elem.type->serializeText(*elem.column, j, out, format_settings);
} }
widths[i][j] = std::min(format_settings.pretty.max_column_pad_width, widths[i][j] = std::min<UInt64>(format_settings.pretty.max_column_pad_width,
UTF8::computeWidth(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size(), prefix)); UTF8::computeWidth(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size(), prefix));
max_widths[i] = std::max(max_widths[i], widths[i][j]); max_widths[i] = std::max(max_widths[i], widths[i][j]);
} }
@ -69,7 +69,7 @@ void PrettyBlockOutputStream::calculateWidths(
/// And also calculate widths for names of columns. /// And also calculate widths for names of columns.
{ {
// name string doesn't contain Tab, no need to pass `prefix` // name string doesn't contain Tab, no need to pass `prefix`
name_widths[i] = std::min(format_settings.pretty.max_column_pad_width, name_widths[i] = std::min<UInt64>(format_settings.pretty.max_column_pad_width,
UTF8::computeWidth(reinterpret_cast<const UInt8 *>(elem.name.data()), elem.name.size())); UTF8::computeWidth(reinterpret_cast<const UInt8 *>(elem.name.data()), elem.name.size()));
max_widths[i] = std::max(max_widths[i], name_widths[i]); max_widths[i] = std::max(max_widths[i], name_widths[i]);
} }

View File

@ -47,9 +47,11 @@ if (ENABLE_TESTS)
endif () endif ()
if (USE_EMBEDDED_COMPILER) if (USE_EMBEDDED_COMPILER)
target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${LLVM_INCLUDE_DIRS}) llvm_libs_all(REQUIRED_LLVM_LIBRARIES)
target_link_libraries(clickhouse_functions PRIVATE ${REQUIRED_LLVM_LIBRARIES})
target_include_directories(clickhouse_functions SYSTEM BEFORE PUBLIC ${LLVM_INCLUDE_DIRS})
endif () endif ()
if (USE_BASE64) if(USE_BASE64)
target_include_directories (clickhouse_functions SYSTEM PRIVATE ${BASE64_INCLUDE_DIR}) target_include_directories(clickhouse_functions SYSTEM PRIVATE ${BASE64_INCLUDE_DIR})
endif() endif()

View File

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

View File

@ -7,7 +7,6 @@
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
@ -118,8 +117,7 @@ NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpressio
} }
else if (table_expression.database_and_table_name) else if (table_expression.database_and_table_name)
{ {
const auto & identifier = static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name); DatabaseAndTableWithAlias database_table(table_expression.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
const auto & table = context.getTable(database_table.database, database_table.table); const auto & table = context.getTable(database_table.database, database_table.table);
names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList(); 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_alias_to_name = data.array_join_alias_to_name;
NameToNameMap & array_join_result_to_source = data.array_join_result_to_source; NameToNameMap & array_join_result_to_source = data.array_join_result_to_source;
if (!node.general()) if (!getColumnIdentifierName(node))
return; return;
auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1 auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1

View File

@ -137,10 +137,7 @@ void AsynchronousMetrics::update()
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER
{ {
if (auto compiled_expression_cache = context.getCompiledExpressionCache()) if (auto compiled_expression_cache = context.getCompiledExpressionCache())
{
set("CompiledExpressionCacheBytes", compiled_expression_cache->weight());
set("CompiledExpressionCacheCount", compiled_expression_cache->count()); set("CompiledExpressionCacheCount", compiled_expression_cache->count());
}
} }
#endif #endif

View File

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

View File

@ -16,18 +16,21 @@ struct ColumnNamesContext
{ {
struct JoinedTable struct JoinedTable
{ {
const ASTTableExpression * expr; const ASTTableExpression * expr = nullptr;
const ASTTableJoin * join; const ASTTableJoin * join = nullptr;
std::optional<String> alias() const std::optional<String> alias() const
{ {
String alias; String alias;
if (expr->database_and_table_name) if (expr)
alias = expr->database_and_table_name->tryGetAlias(); {
else if (expr->table_function) if (expr->database_and_table_name)
alias = expr->table_function->tryGetAlias(); alias = expr->database_and_table_name->tryGetAlias();
else if (expr->subquery) else if (expr->table_function)
alias = expr->subquery->tryGetAlias(); alias = expr->table_function->tryGetAlias();
else if (expr->subquery)
alias = expr->subquery->tryGetAlias();
}
if (!alias.empty()) if (!alias.empty())
return alias; return alias;
return {}; return {};
@ -35,9 +38,8 @@ struct ColumnNamesContext
std::optional<String> name() const std::optional<String> name() const
{ {
if (auto * node = expr->database_and_table_name.get()) if (expr)
if (auto * identifier = typeid_cast<const ASTIdentifier *>(node)) return getIdentifierName(expr->database_and_table_name);
return identifier->name;
return {}; return {};
} }

View File

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

View File

@ -23,6 +23,8 @@ struct DatabaseAndTableWithAlias
String table; String table;
String alias; String alias;
DatabaseAndTableWithAlias() = default;
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = ""); DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, 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; extern const int EXPECTED_ALL_OR_ANY;
} }
/// From SyntaxAnalyzer.cpp
extern void removeDuplicateColumns(NamesAndTypesList & columns);
ExpressionAnalyzer::ExpressionAnalyzer( ExpressionAnalyzer::ExpressionAnalyzer(
const ASTPtr & query_, const ASTPtr & query_,
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_, 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. /// TODO This syntax does not support specifying a database name.
if (table_to_join.database_and_table_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(table_to_join.database_and_table_name);
DatabaseAndTableWithAlias database_table(identifier);
StoragePtr table = context.tryGetTable(database_table.database, database_table.table); StoragePtr table = context.tryGetTable(database_table.database, database_table.table);
if (table) if (table)

View File

@ -189,70 +189,36 @@ auto wrapJITSymbolResolver(llvm::JITSymbolResolver & jsr)
} }
#endif #endif
#if LLVM_VERSION_MAJOR >= 6 #if LLVM_VERSION_MAJOR >= 7
struct CountingMMapper final : public llvm::SectionMemoryManager::MemoryMapper using ModulePtr = std::unique_ptr<llvm::Module>;
{ #else
MemoryTracker memory_tracker{VariableContext::Global}; using ModulePtr = std::shared_ptr<llvm::Module>;
llvm::sys::MemoryBlock allocateMappedMemory(llvm::SectionMemoryManager::AllocationPurpose /*purpose*/,
size_t num_bytes,
const llvm::sys::MemoryBlock * const near_block,
unsigned flags,
std::error_code & error_code) override
{
memory_tracker.alloc(num_bytes);
return llvm::sys::Memory::allocateMappedMemory(num_bytes, near_block, flags, error_code);
}
std::error_code protectMappedMemory(const llvm::sys::MemoryBlock & block, unsigned flags) override
{
return llvm::sys::Memory::protectMappedMemory(block, flags);
}
std::error_code releaseMappedMemory(llvm::sys::MemoryBlock & block) override
{
memory_tracker.free(block.size());
return llvm::sys::Memory::releaseMappedMemory(block);
}
};
#endif #endif
struct LLVMContext struct LLVMContext
{ {
static inline std::atomic<size_t> id_counter{0}; std::shared_ptr<llvm::LLVMContext> context;
llvm::LLVMContext context;
#if LLVM_VERSION_MAJOR >= 7 #if LLVM_VERSION_MAJOR >= 7
llvm::orc::ExecutionSession execution_session; llvm::orc::ExecutionSession execution_session;
std::unique_ptr<llvm::Module> module;
#else
std::shared_ptr<llvm::Module> module;
#endif #endif
ModulePtr module;
std::unique_ptr<llvm::TargetMachine> machine; std::unique_ptr<llvm::TargetMachine> machine;
#if LLVM_VERSION_MAJOR >= 6
std::unique_ptr<CountingMMapper> memory_mapper;
#endif
std::shared_ptr<llvm::SectionMemoryManager> memory_manager; std::shared_ptr<llvm::SectionMemoryManager> memory_manager;
llvm::orc::RTDyldObjectLinkingLayer object_layer; llvm::orc::RTDyldObjectLinkingLayer object_layer;
llvm::orc::IRCompileLayer<decltype(object_layer), llvm::orc::SimpleCompiler> compile_layer; llvm::orc::IRCompileLayer<decltype(object_layer), llvm::orc::SimpleCompiler> compile_layer;
llvm::DataLayout layout; llvm::DataLayout layout;
llvm::IRBuilder<> builder; llvm::IRBuilder<> builder;
std::unordered_map<std::string, void *> symbols; std::unordered_map<std::string, void *> symbols;
size_t id;
LLVMContext() LLVMContext()
: context(std::make_shared<llvm::LLVMContext>())
#if LLVM_VERSION_MAJOR >= 7 #if LLVM_VERSION_MAJOR >= 7
: module(std::make_unique<llvm::Module>("jit", context)) , module(std::make_unique<llvm::Module>("jit", *context))
#else #else
: module(std::make_shared<llvm::Module>("jit", context)) , module(std::make_shared<llvm::Module>("jit", *context))
#endif #endif
, machine(getNativeMachine()) , machine(getNativeMachine())
#if LLVM_VERSION_MAJOR >= 6
, memory_mapper(std::make_unique<CountingMMapper>())
, memory_manager(std::make_shared<llvm::SectionMemoryManager>(memory_mapper.get()))
#else
, memory_manager(std::make_shared<llvm::SectionMemoryManager>()) , memory_manager(std::make_shared<llvm::SectionMemoryManager>())
#endif
#if LLVM_VERSION_MAJOR >= 7 #if LLVM_VERSION_MAJOR >= 7
, object_layer(execution_session, [this](llvm::orc::VModuleKey) , object_layer(execution_session, [this](llvm::orc::VModuleKey)
{ {
@ -263,18 +229,17 @@ struct LLVMContext
#endif #endif
, compile_layer(object_layer, llvm::orc::SimpleCompiler(*machine)) , compile_layer(object_layer, llvm::orc::SimpleCompiler(*machine))
, layout(machine->createDataLayout()) , layout(machine->createDataLayout())
, builder(context) , builder(*context)
, id(id_counter++)
{ {
module->setDataLayout(layout); module->setDataLayout(layout);
module->setTargetTriple(machine->getTargetTriple().getTriple()); module->setTargetTriple(machine->getTargetTriple().getTriple());
} }
/// returns used memory /// returns used memory
size_t compileAllFunctionsToNativeCode() void compileAllFunctionsToNativeCode()
{ {
if (!module->size()) if (!module->size())
return 0; return;
llvm::PassManagerBuilder pass_manager_builder; llvm::PassManagerBuilder pass_manager_builder;
llvm::legacy::PassManager mpm; llvm::legacy::PassManager mpm;
llvm::legacy::FunctionPassManager fpm(module.get()); llvm::legacy::FunctionPassManager fpm(module.get());
@ -323,26 +288,20 @@ struct LLVMContext
throw Exception("Function " + name + " failed to link", ErrorCodes::CANNOT_COMPILE_CODE); throw Exception("Function " + name + " failed to link", ErrorCodes::CANNOT_COMPILE_CODE);
symbols[name] = reinterpret_cast<void *>(*address); symbols[name] = reinterpret_cast<void *>(*address);
} }
#if LLVM_VERSION_MAJOR >= 6
return memory_mapper->memory_tracker.get();
#else
return 0;
#endif
} }
}; };
class LLVMPreparedFunction : public PreparedFunctionImpl class LLVMPreparedFunction : public PreparedFunctionImpl
{ {
std::string name; std::string name;
std::shared_ptr<LLVMContext> context;
void * function; void * function;
public: public:
LLVMPreparedFunction(std::string name_, std::shared_ptr<LLVMContext> context) LLVMPreparedFunction(const std::string & name_, const std::unordered_map<std::string, void *> & symbols)
: name(std::move(name_)), context(context) : name(name_)
{ {
auto it = context->symbols.find(name); auto it = symbols.find(name);
if (context->symbols.end() == it) if (symbols.end() == it)
throw Exception("Cannot find symbol " + name + " in LLVMContext", ErrorCodes::LOGICAL_ERROR); throw Exception("Cannot find symbol " + name + " in LLVMContext", ErrorCodes::LOGICAL_ERROR);
function = it->second; function = it->second;
} }
@ -373,16 +332,16 @@ public:
} }
}; };
static void compileFunctionToLLVMByteCode(std::shared_ptr<LLVMContext> & context, const IFunctionBase & f) static void compileFunctionToLLVMByteCode(LLVMContext & context, const IFunctionBase & f)
{ {
ProfileEvents::increment(ProfileEvents::CompileFunction); ProfileEvents::increment(ProfileEvents::CompileFunction);
auto & arg_types = f.getArgumentTypes(); auto & arg_types = f.getArgumentTypes();
auto & b = context->builder; auto & b = context.builder;
auto * size_type = b.getIntNTy(sizeof(size_t) * 8); auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy(), size_type); auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy(), size_type);
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false); auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), context->module.get()); auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), context.module.get());
auto args = func->args().begin(); auto args = func->args().begin();
llvm::Value * counter_arg = &*args++; llvm::Value * counter_arg = &*args++;
llvm::Value * columns_arg = &*args++; llvm::Value * columns_arg = &*args++;
@ -504,12 +463,21 @@ static CompilableExpression subexpression(const IFunctionBase & f, std::vector<C
}; };
} }
LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, std::shared_ptr<LLVMContext> context, const Block & sample_block) struct LLVMModuleState
: name(actions.back().result_name), context(context)
{ {
std::unordered_map<std::string, void *> symbols;
std::shared_ptr<llvm::LLVMContext> major_context;
std::shared_ptr<llvm::SectionMemoryManager> memory_manager;
};
LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const Block & sample_block)
: name(actions.back().result_name)
, module_state(std::make_unique<LLVMModuleState>())
{
LLVMContext context;
for (const auto & c : sample_block) for (const auto & c : sample_block)
/// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)` /// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)`
if (c.column && getNativeValue(toNativeType(context->builder, c.type), *c.column, 0)) if (c.column && getNativeValue(toNativeType(context.builder, c.type), *c.column, 0))
subexpressions[c.name] = subexpression(c.column, c.type); subexpressions[c.name] = subexpression(c.column, c.type);
for (const auto & action : actions) for (const auto & action : actions)
{ {
@ -530,6 +498,11 @@ LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, std::shar
originals.push_back(action.function_base); originals.push_back(action.function_base);
} }
compileFunctionToLLVMByteCode(context, *this); compileFunctionToLLVMByteCode(context, *this);
context.compileAllFunctionsToNativeCode();
module_state->symbols = context.symbols;
module_state->major_context = context.context;
module_state->memory_manager = context.memory_manager;
} }
llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const
@ -540,8 +513,7 @@ llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlacehol
return it->second(builder, values); return it->second(builder, values);
} }
PreparedFunctionPtr LLVMFunction::prepare(const Block &, const ColumnNumbers &, size_t) const { return std::make_shared<LLVMPreparedFunction>(name, module_state->symbols); }
PreparedFunctionPtr LLVMFunction::prepare(const Block &, const ColumnNumbers &, size_t) const { return std::make_shared<LLVMPreparedFunction>(name, context); }
bool LLVMFunction::isDeterministic() const bool LLVMFunction::isDeterministic() const
{ {
@ -622,28 +594,6 @@ static bool isCompilable(const IFunctionBase & function)
return function.isCompilable(); return function.isCompilable();
} }
size_t CompiledExpressionCache::weight() const
{
#if LLVM_VERSION_MAJOR >= 6
std::lock_guard lock(mutex);
size_t result{0};
std::unordered_set<size_t> seen;
for (const auto & cell : cells)
{
auto function_context = cell.second.value->getContext();
if (!seen.count(function_context->id))
{
result += function_context->memory_mapper->memory_tracker.get();
seen.insert(function_context->id);
}
}
return result;
#else
return Base::weight();
#endif
}
std::vector<std::unordered_set<std::optional<size_t>>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns) std::vector<std::unordered_set<std::optional<size_t>>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns)
{ {
/// an empty optional is a poisoned value prohibiting the column's producer from being removed /// an empty optional is a poisoned value prohibiting the column's producer from being removed
@ -748,21 +698,16 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] () std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] ()
{ {
Stopwatch watch; Stopwatch watch;
std::shared_ptr<LLVMContext> context = std::make_shared<LLVMContext>(); std::shared_ptr<LLVMFunction> result_fn;
auto result_fn = std::make_shared<LLVMFunction>(inlined_func, context, sample_block); result_fn = std::make_shared<LLVMFunction>(inlined_func, sample_block);
size_t used_memory = context->compileAllFunctionsToNativeCode();
ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory);
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
return result_fn; return result_fn;
}); });
} }
else else
{ {
std::shared_ptr<LLVMContext> context = std::make_shared<LLVMContext>();
Stopwatch watch; Stopwatch watch;
fn = std::make_shared<LLVMFunction>(fused[i], context, sample_block); fn = std::make_shared<LLVMFunction>(fused[i], sample_block);
size_t used_memory = context->compileAllFunctionsToNativeCode();
ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory);
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
} }

View File

@ -14,19 +14,23 @@
namespace DB namespace DB
{ {
struct LLVMContext;
using CompilableExpression = std::function<llvm::Value * (llvm::IRBuilderBase &, const ValuePlaceholders &)>; using CompilableExpression = std::function<llvm::Value * (llvm::IRBuilderBase &, const ValuePlaceholders &)>;
struct LLVMModuleState;
class LLVMFunction : public IFunctionBase class LLVMFunction : public IFunctionBase
{ {
std::string name; std::string name;
Names arg_names; Names arg_names;
DataTypes arg_types; DataTypes arg_types;
std::shared_ptr<LLVMContext> context;
std::vector<FunctionBasePtr> originals; std::vector<FunctionBasePtr> originals;
std::unordered_map<StringRef, CompilableExpression> subexpressions; std::unordered_map<StringRef, CompilableExpression> subexpressions;
std::unique_ptr<LLVMModuleState> module_state;
public: public:
LLVMFunction(const ExpressionActions::Actions & actions, std::shared_ptr<LLVMContext> context, const Block & sample_block); LLVMFunction(const ExpressionActions::Actions & actions, const Block & sample_block);
bool isCompilable() const override { return true; } bool isCompilable() const override { return true; }
@ -54,8 +58,7 @@ public:
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override; Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override;
std::shared_ptr<LLVMContext> getContext() const { return context; } const LLVMModuleState * getLLVMModuleState() const { return module_state.get(); }
}; };
/** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion. /** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion.
@ -63,13 +66,9 @@ public:
*/ */
class CompiledExpressionCache : public LRUCache<UInt128, LLVMFunction, UInt128Hash> class CompiledExpressionCache : public LRUCache<UInt128, LLVMFunction, UInt128Hash>
{ {
private:
using Base = LRUCache<UInt128, LLVMFunction, UInt128Hash>;
public: public:
using Base = LRUCache<UInt128, LLVMFunction, UInt128Hash>;
using Base::Base; using Base::Base;
size_t weight() const;
}; };
/// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable /// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable

View File

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

View File

@ -112,7 +112,7 @@ public:
* instead of doing a subquery, you just need to read it. * 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())) 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/Context.h>
#include <Interpreters/DatabaseAndTableWithAlias.h> #include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Storages/StorageDistributed.h> #include <Storages/StorageDistributed.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.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) StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & context)
{ {
const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(database_and_table.get()); DatabaseAndTableWithAlias db_and_table(database_and_table);
if (!id)
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
DatabaseAndTableWithAlias db_and_table(*id);
return context.tryGetTable(db_and_table.database, db_and_table.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::string table;
std::tie(database, table) = getRemoteDatabaseAndTableName(*storage); std::tie(database, table) = getRemoteDatabaseAndTableName(*storage);
/// TODO: find a way to avoid AST node replacing database_and_table = createTableIdentifier(database, table);
database_and_table = createDatabaseAndTableNode(database, table);
} }
else else
throw Exception("InJoinSubqueriesPreprocessor: unexpected value of 'distributed_product_mode' setting", ErrorCodes::LOGICAL_ERROR); throw Exception("InJoinSubqueriesPreprocessor: unexpected value of 'distributed_product_mode' setting", ErrorCodes::LOGICAL_ERROR);

View File

@ -195,9 +195,11 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col
{ {
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast); auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
DataTypePtr column_type = nullptr;
if (col_decl.type) if (col_decl.type)
{ {
columns.emplace_back(col_decl.name, DataTypeFactory::instance().get(col_decl.type)); column_type = DataTypeFactory::instance().get(col_decl.type);
columns.emplace_back(col_decl.name, column_type);
} }
else else
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions /// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
@ -228,7 +230,7 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col
if (col_decl.codec) if (col_decl.codec)
{ {
auto codec = CompressionCodecFactory::instance().get(col_decl.codec); auto codec = CompressionCodecFactory::instance().get(col_decl.codec, column_type);
codecs.emplace(col_decl.name, codec); codecs.emplace(col_decl.name, codec);
} }

View File

@ -95,13 +95,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
auto database_ptr = identifier->children[0]; auto database_ptr = identifier->children[0];
auto table_ptr = identifier->children[1]; auto table_ptr = identifier->children[1];
if (database_ptr) getIdentifierName(database_ptr, database_name);
database_name = typeid_cast<ASTIdentifier &>(*database_ptr).name; getIdentifierName(table_ptr, table_name);
if (table_ptr)
table_name = typeid_cast<ASTIdentifier &>(*table_ptr).name;
} }
else else
table_name = typeid_cast<ASTIdentifier &>(*identifier).name; getIdentifierName(identifier, table_name);
table = context.getTable(database_name, 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()) if (!alias.empty())
hidden.push_back(alias); hidden.push_back(alias);
if (auto * identifier = typeid_cast<const ASTIdentifier *>(table_expression->children[0].get())) if (auto opt_name = getIdentifierName(table_expression->children[0]))
hidden.push_back(identifier->name); hidden.push_back(*opt_name);
else if (alias.empty()) else if (alias.empty())
throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR); 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::Data qn_visitor_data{{}, tables};
TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast); TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast);
QueryAliasesVisitor::Data query_aliases_data{aliases}; QueryAliasesVisitor::Data query_aliases_data{aliases};
QueryAliasesVisitor(query_aliases_data).visit(ast); 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) for (const auto & projection_column : select_query->select_expression_list->children)
{ {

View File

@ -20,39 +20,253 @@ namespace ErrorCodes
extern const int CYCLIC_ALIASES; extern const int CYCLIC_ALIASES;
} }
class CheckASTDepth
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)
{ {
} public:
CheckASTDepth(QueryNormalizer::Data & data_)
void QueryNormalizer::perform() : data(data_)
{
SetOfASTs tmp_set;
MapOfASTs tmp_map;
performImpl(query, tmp_map, tmp_set, "", 0);
try
{ {
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)"); --data.level;
throw; }
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) void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
/// 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)
{ {
if (level > settings.max_ast_depth) auto & current_asts = data.current_asts;
throw Exception("Normalized AST is too deep. Maximum: " + toString(settings.max_ast_depth), ErrorCodes::TOO_DEEP_AST); 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)) if (finished_asts.count(ast))
{ {
@ -63,185 +277,46 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
ASTPtr initial_ast = ast; ASTPtr initial_ast = ast;
current_asts.insert(initial_ast.get()); 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)`. String my_alias = ast->tryGetAlias();
if (functionIsInOrGlobalInOperator(func_node->name)) if (!my_alias.empty())
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(func_node->arguments->children.at(1).get())) data.current_alias = my_alias;
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();
}
}
} }
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()) if (auto * node = typeid_cast<ASTFunction *>(ast.get()))
{ visit(*node, ast, data);
/// Avoid infinite recursion here if (auto * node = typeid_cast<ASTIdentifier *>(ast.get()))
auto replace_to_identifier = typeid_cast<ASTIdentifier *>(it_alias->second.get()); visit(*node, ast, data);
bool is_cycle = replace_to_identifier && replace_to_identifier->general() if (auto * node = typeid_cast<ASTExpressionList *>(ast.get()))
&& replace_to_identifier->name == identifier_node->name; visit(*node, ast, data);
if (auto * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
if (!is_cycle) visit(*node, ast, data);
{ if (auto * node = typeid_cast<ASTSelectQuery *>(ast.get()))
/// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. visit(*node, ast, data);
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 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 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) if (ast.get() != initial_ast.get())
{ visit(ast, data);
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)
{
}
else else
{ visitChildren(ast, data);
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);
}
current_asts.erase(initial_ast.get()); current_asts.erase(initial_ast.get());
current_asts.erase(ast.get()); current_asts.erase(ast.get());
finished_asts[initial_ast] = ast; 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>; class ASTFunction;
using TableNamesAndColumnNames = std::vector<TableNameAndColumnNames>; class ASTIdentifier;
class ASTExpressionList;
struct ASTTablesInSelectQueryElement;
class QueryNormalizer class QueryNormalizer
@ -41,23 +43,54 @@ class QueryNormalizer
public: public:
using Aliases = std::unordered_map<String, ASTPtr>; 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, struct Data
const TableNamesAndColumnNames & table_names_and_column_names); {
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: private:
using SetOfASTs = std::set<const IAST *>; Data & visitor_data;
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
ASTPtr & query; static void visit(ASTPtr & query, Data & data);
const Aliases & aliases;
const ExtractedSettings settings;
const Names & all_column_names;
const TableNamesAndColumnNames & table_names_and_column_names;
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; 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();) for (auto it = columns.begin(); it != columns.end();)
{ {
if (names.emplace(it->name).second) if (names.emplace(it->name).second)
@ -52,6 +52,7 @@ void removeDuplicateColumns(NamesAndTypesList & columns)
else else
columns.erase(it++); columns.erase(it++);
} }
return names;
} }
namespace 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()); 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. /// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names.
@ -102,12 +101,11 @@ void normalizeTree(
SyntaxAnalyzerResult & result, SyntaxAnalyzerResult & result,
const Names & source_columns, const Names & source_columns,
const NameSet & source_columns_set, const NameSet & source_columns_set,
const StoragePtr & storage,
const Context & context, const Context & context,
const ASTSelectQuery * select_query, const ASTSelectQuery * select_query,
bool asterisk_left_columns_only) 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) if (!asterisk_left_columns_only)
{ {
@ -119,17 +117,20 @@ void normalizeTree(
if (all_columns_name.empty()) if (all_columns_name.empty())
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); 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()) if (select_query && select_query->tables && !select_query->tables->children.empty())
{ {
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query); std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
bool first = true; bool first = true;
String current_database = context.getCurrentDatabase();
for (const auto * table_expression : tables_expression) 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); NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context);
removeDuplicateColumns(names_and_types);
if (!first) if (!first)
{ {
/// For joined tables qualify duplicating names. /// For joined tables qualify duplicating names.
@ -140,12 +141,14 @@ void normalizeTree(
first = false; 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::Data normalizer_data(result.aliases, context.getSettingsRef(), std::move(table_with_columns));
QueryNormalizer(query, result.aliases, settings, all_columns_name, table_names_and_column_names).perform(); QueryNormalizer(normalizer_data).visit(query);
} }
bool hasArrayJoin(const ASTPtr & ast) 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_name = ast->getColumnName();
const String nested_table_alias = ast->getAliasOrColumnName(); 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, throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name,
ErrorCodes::ALIAS_REQUIRED); ErrorCodes::ALIAS_REQUIRED);
@ -468,7 +471,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
String result_name = expr->getAliasOrColumnName(); String result_name = expr->getAliasOrColumnName();
/// This is an array. /// 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; 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; std::function<TableBelonging(const ASTPtr &)> get_table_belonging;
get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging
{ {
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get()); if (getColumnIdentifierName(ast))
if (identifier)
{ {
if (identifier->general()) auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
{ {
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names); auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_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; std::function<void(ASTPtr &, const DatabaseAndTableWithAlias &, bool)> translate_qualified_names;
translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table) 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); auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
stripIdentifier(ast, num_components); stripIdentifier(ast, num_components);
@ -739,6 +743,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
result.source_columns = source_columns_; result.source_columns = source_columns_;
collectSourceColumns(select_query, result.storage, result.source_columns); collectSourceColumns(select_query, result.storage, result.source_columns);
NameSet source_columns_set = removeDuplicateColumns(result.source_columns);
const auto & settings = context.getSettingsRef(); const auto & settings = context.getSettingsRef();
@ -746,7 +751,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
source_columns_list.reserve(result.source_columns.size()); source_columns_list.reserve(result.source_columns.size());
for (const auto & type_name : result.source_columns) for (const auto & type_name : result.source_columns)
source_columns_list.emplace_back(type_name.name); 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) if (select_query)
{ {
@ -768,7 +775,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
} }
/// Common subexpression elimination. Rewrite rules. /// 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); context, select_query, settings.asterisk_left_columns_only != 0);
/// Remove unneeded columns according to 'required_result_columns'. /// Remove unneeded columns according to 'required_result_columns'.

View File

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

View File

@ -55,7 +55,7 @@ std::vector<ASTPtr *> TranslateQualifiedNamesMatcher::visit(const ASTIdentifier
const NameSet & source_columns = data.source_columns; const NameSet & source_columns = data.source_columns;
const std::vector<DatabaseAndTableWithAlias> & tables = data.tables; 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. /// Select first table name with max number of qualifiers which can be stripped.
size_t max_num_qualifiers_to_strip = 0; 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) 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(); : context.getDefaultFormat();
BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader()); BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader());

View File

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

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <IO/WriteBufferFromOStream.h> #include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -38,4 +39,82 @@ void ASTIdentifier::appendColumnNameImpl(WriteBuffer & ostr) const
writeString(name, ostr); 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 #pragma once
#include <optional>
#include <Parsers/ASTWithAlias.h> #include <Parsers/ASTWithAlias.h>
namespace DB namespace DB
{ {
/** Identifier (column or alias) /// Identifier (column, table or alias)
*/
class ASTIdentifier : public ASTWithAlias class ASTIdentifier : public ASTWithAlias
{ {
enum Kind /// TODO This is semantic, not syntax. Remove it.
{
General,
Special, // Database, Table, Format
};
public: 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. /// 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; String name;
ASTIdentifier(const String & name_, const Kind kind_ = General) ASTIdentifier(const String & name_)
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); } : name(name_)
, special(false)
{
range = StringRange(name.data(), name.data() + name.size());
}
/** Get the text that identifies this element. */ /** Get the text that identifies this element. */
String getID(char delim) const override { return "Identifier" + (delim + name); } String getID(char delim) const override { return "Identifier" + (delim + name); }
@ -33,21 +32,50 @@ public:
set.insert(name); 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: protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override;
private: 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; 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 ASTPtr ASTSelectQuery::clone() const
{ {
@ -338,7 +325,7 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
} }
String table_alias = getTableExpressionAlias(table_expression); 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()) if (!table_alias.empty())
table_expression->database_and_table_name->setAlias(table_alias); 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; 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/ReadHelpers.h>
#include <IO/ReadBufferFromMemory.h> #include <IO/ReadBufferFromMemory.h>
#include <Common/typeid_cast.h>
#include <Parsers/DumpASTNode.h> #include <Parsers/DumpASTNode.h>
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
@ -173,7 +174,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
{ {
if (!name.empty()) if (!name.empty())
name += '.'; name += '.';
name += static_cast<const ASTIdentifier &>(*child.get()).name; name += *getIdentifierName(child);
} }
node = std::make_shared<ASTIdentifier>(name); 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, * 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. * 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_end - contents_begin == strlen("2014-01-01")
&& contents_begin[0] >= '2' && contents_begin[0] <= '3' && contents_begin[0] >= '2' && contents_begin[0] <= '3'
&& contents_begin[1] >= '0' && contents_begin[1] <= '9' && 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>(); 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(...) /// func(DISTINCT ...) is equivalent to funcDistinct(...)
if (has_distinct_modifier) 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. * 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) for (const char ** keyword = restricted_keywords; *keyword != nullptr; ++keyword)
if (0 == strcasecmp(name.data(), *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; bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword;
if (allow_alias_without_as_keyword) if (allow_alias_without_as_keyword)
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(node.get())) if (auto opt_id = getIdentifierName(node))
if (0 == strcasecmp(id->name.data(), "FROM")) if (0 == strcasecmp(opt_id->data(), "FROM"))
allow_alias_without_as_keyword_now = false; allow_alias_without_as_keyword_now = false;
ASTPtr alias_node; ASTPtr alias_node;
if (ParserAlias(allow_alias_without_as_keyword_now).parse(pos, alias_node, expected)) 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())) 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; ast_with_alias->prefer_alias_to_column_name = prefer_alias_to_column_name;
} }
else else

View File

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

View File

@ -1,3 +1,4 @@
#include <Common/typeid_cast.h>
#include <Parsers/ParserAlterQuery.h> #include <Parsers/ParserAlterQuery.h>
#include <Parsers/CommonParsers.h> #include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.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)) if (!p_expression.parse(pos, assignment->expression, expected))
return false; return false;
assignment->column_name = typeid_cast<const ASTIdentifier &>(*column).name; getIdentifierName(column, assignment->column_name);
if (assignment->expression) if (assignment->expression)
assignment->children.push_back(assignment->expression); assignment->children.push_back(assignment->expression);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -5,8 +5,6 @@
#include <Parsers/ASTOptimizeQuery.h> #include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
namespace DB namespace DB
{ {
@ -60,10 +58,8 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
auto query = std::make_shared<ASTOptimizeQuery>(); auto query = std::make_shared<ASTOptimizeQuery>();
node = query; node = query;
if (database) getIdentifierName(database, query->database);
query->database = typeid_cast<const ASTIdentifier &>(*database).name; getIdentifierName(table, query->table);
if (table)
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
query->cluster = cluster_str; query->cluster = cluster_str;
query->partition = partition; 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)) if (!format_p.parse(pos, query_with_output.format, expected))
return false; 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); query_with_output.children.push_back(query_with_output.format);
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -14,6 +14,7 @@
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h> #include <Parsers/ASTColumnDeclaration.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Compression/CompressionFactory.h>
namespace DB namespace DB
@ -30,6 +31,7 @@ namespace ErrorCodes
std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast) std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast)
{ {
const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
const CompressionCodecFactory & compression_codec_factory = CompressionCodecFactory::instance();
if (command_ast->type == ASTAlterCommand::ADD_COLUMN) 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; 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) 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; command.if_not_exists = command_ast->if_not_exists;
@ -63,7 +68,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
AlterCommand command; AlterCommand command;
command.type = AlterCommand::DROP_COLUMN; 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; command.if_exists = command_ast->if_exists;
return command; return command;
} }
@ -86,6 +91,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.default_expression = ast_col_decl.default_expression; 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) if (ast_col_decl.comment)
{ {
const auto & ast_comment = typeid_cast<ASTLiteral &>(*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; AlterCommand command;
command.type = COMMENT_COLUMN; command.type = COMMENT_COLUMN;
const auto & ast_identifier = typeid_cast<ASTIdentifier &>(*command_ast->column); command.column_name = *getIdentifierName(command_ast->column);
command.column_name = ast_identifier.name;
const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment); const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment);
command.comment = ast_comment.value.get<String>(); command.comment = ast_comment.value.get<String>();
command.if_exists = command_ast->if_exists; command.if_exists = command_ast->if_exists;
@ -169,6 +176,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
if (default_expression) if (default_expression)
columns_description.defaults.emplace(column_name, ColumnDefault{default_kind, 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 /// Slow, because each time a list is copied
columns_description.ordinary = Nested::flatten(columns_description.ordinary); 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) else if (type == MODIFY_COLUMN)
{ {
if (codec)
columns_description.codecs[column_name] = codec;
if (!is_mutable()) if (!is_mutable())
{ {
auto & comments = columns_description.comments; 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. /// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
bool ignore = false; bool ignore = false;
/// For ADD and MODIFY
CompressionCodecPtr codec;
AlterCommand() = default; AlterCommand() = default;
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
const ColumnDefaultKind default_kind, const ASTPtr & default_expression, const ColumnDefaultKind default_kind, const ASTPtr & default_expression,

View File

@ -1,40 +1,34 @@
#include <Common/config.h> #include <Storages/Kafka/StorageKafka.h>
#include <Common/config_version.h>
#if USE_RDKAFKA #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/IProfilingBlockInputStream.h>
#include <DataStreams/LimitBlockInputStream.h> #include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/UnionBlockInputStream.h> #include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/copyData.h> #include <DataStreams/copyData.h>
#include <Formats/FormatFactory.h>
#include <IO/ReadBuffer.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/InterpreterInsertQuery.h> #include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTInsertQuery.h> #include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/Kafka/KafkaSettings.h> #include <Storages/Kafka/KafkaSettings.h>
#include <Storages/Kafka/StorageKafka.h>
#include <Storages/StorageMaterializedView.h>
#include <Storages/StorageFactory.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> #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 namespace DB
{ {
@ -62,8 +56,8 @@ static const String CONFIG_PREFIX = "kafka";
class ReadBufferFromKafkaConsumer : public ReadBuffer class ReadBufferFromKafkaConsumer : public ReadBuffer
{ {
rd_kafka_t * consumer; ConsumerPtr consumer;
rd_kafka_message_t * current = nullptr; cppkafka::Message current;
bool current_pending = false; /// We've fetched "current" message and need to process it on the next iteration. bool current_pending = false; /// We've fetched "current" message and need to process it on the next iteration.
Poco::Logger * log; Poco::Logger * log;
size_t read_messages = 0; size_t read_messages = 0;
@ -73,42 +67,36 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer
{ {
if (current_pending) 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; current_pending = false;
return true; return true;
} }
// Process next buffered message // Process next buffered message
rd_kafka_message_t * msg = rd_kafka_consumer_poll(consumer, READ_POLL_MS); // XXX: use RAII. auto message = consumer->poll(std::chrono::milliseconds(READ_POLL_MS));
if (msg == nullptr) if (!message)
return false; return false;
if (msg->err) if (message.is_eof())
{ {
if (msg->err != RD_KAFKA_RESP_ERR__PARTITION_EOF) // Reached EOF while reading current batch, skip it.
{ LOG_TRACE(log, "EOF reached for partition " << message.get_partition() << " offset " << message.get_offset());
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);
return nextImpl(); return nextImpl();
} }
else if (auto err = message.get_error())
if (msg->len && !msg->payload) {
throw Exception("Logical error: nullptr message returned with non-zero length", ErrorCodes::LOGICAL_ERROR); LOG_ERROR(log, "Consumer error: " << err);
return false;
}
++read_messages; ++read_messages;
// Now we've received a new message. Check if we need to produce a delimiter // 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); BufferBase::set(&row_delimiter, 1, 0);
reset(); current = std::move(message);
current = msg;
current_pending = true; current_pending = true;
return true; return true;
} }
@ -116,31 +104,21 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer
// Consume message and mark the topic/partition offset // Consume message and mark the topic/partition offset
// The offsets will be committed in the readSuffix() method after the block is completed // 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 // If an exception is thrown before that would occur, the client will rejoin without committing offsets
reset(); current = std::move(message);
current = msg;
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);
return true; return true;
} }
void reset()
{
if (current != nullptr)
{
rd_kafka_message_destroy(current);
current = nullptr;
}
}
public: 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_) : ReadBuffer(nullptr, 0), consumer(consumer_), log(log_), row_delimiter(row_delimiter_)
{ {
if (row_delimiter != '\0') if (row_delimiter != '\0')
LOG_TRACE(log, "Row delimiter is: " << row_delimiter); LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
} }
~ReadBufferFromKafkaConsumer() override { reset(); }
/// Commit messages read with this consumer /// Commit messages read with this consumer
void commit() void commit()
{ {
@ -148,10 +126,7 @@ public:
if (read_messages == 0) if (read_messages == 0)
return; return;
auto err = rd_kafka_commit(consumer, nullptr, 1 /* async */); consumer->async_commit();
if (err)
throw Exception("Failed to commit offsets: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
read_messages = 0; read_messages = 0;
} }
}; };
@ -215,7 +190,7 @@ public:
if (consumer == nullptr) if (consumer == nullptr)
throw Exception("Failed to claim consumer: ", ErrorCodes::TIMEOUT_EXCEEDED); 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); reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size);
} }
@ -239,7 +214,7 @@ public:
private: private:
StorageKafka & storage; StorageKafka & storage;
StorageKafka::ConsumerPtr consumer; ConsumerPtr consumer;
Context context; Context context;
size_t max_block_size; size_t max_block_size;
Block sample_block; Block sample_block;
@ -251,7 +226,7 @@ private:
bool hasClaimed() { return consumer != nullptr; } 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; AbstractConfiguration::Keys keys;
std::vector<char> errstr(512); 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_path = path + "." + key;
const String key_name = boost::replace_all_copy(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) conf.set(key_name, config.getString(key_path));
throw Exception("Invalid Kafka setting " + key_path + " in config: " + String(errstr.data()), ErrorCodes::INVALID_CONFIG_PARAMETER);
} }
} }
@ -326,21 +300,8 @@ void StorageKafka::startup()
{ {
for (size_t i = 0; i < num_consumers; ++i) 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 // Create a consumer and subscribe to topics
// Note: consumer takes ownership of the configuration auto consumer = std::make_shared<cppkafka::Consumer>(createConsumerConfiguration());
auto consumer = std::make_shared<StorageKafka::Consumer>(consumer_conf);
consumer->subscribe(topics); consumer->subscribe(topics);
// Make consumer available // Make consumer available
@ -362,7 +323,7 @@ void StorageKafka::shutdown()
for (size_t i = 0; i < num_created_consumers; ++i) for (size_t i = 0; i < num_created_consumers; ++i)
{ {
auto consumer = claimConsumer(); 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"); 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); 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) conf.set("metadata.broker.list", brokers);
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
LOG_TRACE(log, "Setting Group ID: " << group << " Client ID: clickhouse"); 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) conf.set("client.id", VERSION_FULL);
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);
// We manually commit offsets after a stream successfully finished // 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 // Update consumer configuration from the configuration
const auto & config = global_context.getConfigRef(); 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)) if (config.has(topic_config_key))
loadFromConfig(conf, config, topic_config_key); loadFromConfig(conf, config, topic_config_key);
} }
return conf;
} }
StorageKafka::ConsumerPtr StorageKafka::claimConsumer() ConsumerPtr StorageKafka::claimConsumer()
{ {
return tryClaimConsumer(-1L); return tryClaimConsumer(-1L);
} }
StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms) ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
{ {
// Wait for the first free consumer // Wait for the first free consumer
if (wait_ms >= 0) if (wait_ms >= 0)
@ -434,7 +393,7 @@ StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
return consumer; return consumer;
} }
void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr consumer) void StorageKafka::pushConsumer(ConsumerPtr consumer)
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
consumers.push_back(consumer); 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) void registerStorageKafka(StorageFactory & factory)
{ {
factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args) factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args)

View File

@ -1,24 +1,24 @@
#pragma once #pragma once
#include <Common/config.h> #include <Common/config.h>
#if USE_RDKAFKA #if USE_RDKAFKA
#include <mutex>
#include <ext/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <Core/BackgroundSchedulePool.h> #include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h> #include <Core/NamesAndTypes.h>
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <Storages/IStorage.h>
#include <Poco/Event.h> #include <Poco/Event.h>
#include <Poco/Semaphore.h> #include <Poco/Semaphore.h>
#include <ext/shared_ptr_helper.h>
struct rd_kafka_s; #include <cppkafka/cppkafka.h>
struct rd_kafka_conf_s; #include <mutex>
namespace DB 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, /** 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. * or as a basic building block for creating pipelines with a continuous insertion / ETL.
@ -53,22 +53,6 @@ public:
void updateDependencies() override; void updateDependencies() override;
private: 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 // Configuration and state
String table_name; String table_name;
String database_name; String database_name;
@ -100,7 +84,7 @@ private:
BackgroundSchedulePool::TaskHolder task; BackgroundSchedulePool::TaskHolder task;
std::atomic<bool> stream_cancelled{false}; std::atomic<bool> stream_cancelled{false};
void consumerConfiguration(struct rd_kafka_conf_s * conf); cppkafka::Configuration createConsumerConfiguration();
ConsumerPtr claimConsumer(); ConsumerPtr claimConsumer();
ConsumerPtr tryClaimConsumer(long wait_ms); ConsumerPtr tryClaimConsumer(long wait_ms);
void pushConsumer(ConsumerPtr c); void pushConsumer(ConsumerPtr c);

View File

@ -91,7 +91,7 @@ Block MergeTreeBaseSelectBlockInputStream::readFromPart()
UInt64 rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes); UInt64 rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes);
if (!rows_to_read) if (!rows_to_read)
return rows_to_read; return rows_to_read;
rows_to_read = std::max(index_granularity, rows_to_read); rows_to_read = std::max<UInt64>(index_granularity, rows_to_read);
if (current_preferred_max_column_in_block_size_bytes) if (current_preferred_max_column_in_block_size_bytes)
{ {

View File

@ -121,7 +121,7 @@ void MergeTreeWhereOptimizer::optimizeConjunction(ASTSelectQuery & select, ASTFu
SCOPE_EXIT(++idx); SCOPE_EXIT(++idx);
if (cannotBeMoved(condition)) if (cannotBeMoved(conditions[idx]))
continue; continue;
IdentifierNameSet identifiers{}; IdentifierNameSet identifiers{};
@ -193,7 +193,7 @@ void MergeTreeWhereOptimizer::optimizeArbitrary(ASTSelectQuery & select) const
auto & condition = select.where_expression; auto & condition = select.where_expression;
/// do not optimize restricted expressions /// do not optimize restricted expressions
if (cannotBeMoved(select.where_expression.get())) if (cannotBeMoved(select.where_expression))
return; return;
IdentifierNameSet identifiers{}; IdentifierNameSet identifiers{};
@ -250,10 +250,10 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const
auto right_arg = function->arguments->children.back().get(); auto right_arg = function->arguments->children.back().get();
/// try to ensure left_arg points to ASTIdentifier /// 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); 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 /// 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)) 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) void MergeTreeWhereOptimizer::collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set)
{ {
if (const auto identifier = typeid_cast<const ASTIdentifier *>(ast)) if (auto opt_name = getIdentifierName(ast))
return (void) set.insert(identifier->name); return (void) set.insert(*opt_name);
if (typeid_cast<const ASTSubquery *>(ast)) if (typeid_cast<const ASTSubquery *>(ast))
return; 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 /// disallow arrayJoin expressions to be moved to PREWHERE for now
if (array_join_function_name == function_ptr->name) if (array_join_function_name == function_ptr->name)
@ -381,17 +381,16 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
if ("indexHint" == function_ptr->name) if ("indexHint" == function_ptr->name)
return true; 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 /// disallow moving result of ARRAY JOIN to PREWHERE
if (identifier_ptr->general()) if (array_joined_names.count(*opt_name) ||
if (array_joined_names.count(identifier_ptr->name) || array_joined_names.count(Nested::extractTableName(*opt_name)))
array_joined_names.count(Nested::extractTableName(identifier_ptr->name))) return true;
return true;
} }
for (const auto & child : ptr->children) for (const auto & child : ptr->children)
if (cannotBeMoved(child.get())) if (cannotBeMoved(child))
return true; return true;
return false; return false;

View File

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

View File

@ -199,7 +199,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
min_saved_log_pointer = std::min(min_saved_log_pointer, min_log_pointer_lost_candidate); min_saved_log_pointer = std::min(min_saved_log_pointer, min_log_pointer_lost_candidate);
/// We will not touch the last `min_replicated_logs_to_keep` records. /// We will not touch the last `min_replicated_logs_to_keep` records.
entries.erase(entries.end() - std::min(entries.size(), storage.data.settings.min_replicated_logs_to_keep.value), entries.end()); entries.erase(entries.end() - std::min<UInt64>(entries.size(), storage.data.settings.min_replicated_logs_to_keep.value), entries.end());
/// We will not touch records that are no less than `min_saved_log_pointer`. /// We will not touch records that are no less than `min_saved_log_pointer`.
entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_saved_log_pointer)), entries.end()); entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_saved_log_pointer)), entries.end());
@ -295,7 +295,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
/// Virtual node, all nodes that are "greater" than this one will be deleted /// Virtual node, all nodes that are "greater" than this one will be deleted
NodeWithStat block_threshold{{}, time_threshold}; 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_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_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); 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; Names res;
res.reserve(elements.size()); res.reserve(elements.size());
for (const auto & elem : elements) for (const auto & elem : elements)
res.push_back(typeid_cast<const ASTIdentifier &>(*elem).name); res.push_back(*getIdentifierName(elem));
return res; return res;
} }
else 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 (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
{ {
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get())) if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
merging_params.sign_column = ast->name;
else
throw Exception( throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); 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 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 (!engine_args.empty() && !typeid_cast<const ASTLiteral *>(engine_args.back().get()))
{ {
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get())) if (!getIdentifierName(engine_args.back(), merging_params.version_column))
merging_params.version_column = ast->name;
else
throw Exception( throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
@ -535,18 +531,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
} }
else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
{ {
if (auto ast = typeid_cast<ASTIdentifier *>(engine_args.back().get())) if (!getIdentifierName(engine_args.back(), merging_params.version_column))
merging_params.version_column = ast->name;
else
throw Exception( throw Exception(
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
engine_args.pop_back(); engine_args.pop_back();
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get())) if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
merging_params.sign_column = ast->name;
else
throw Exception( throw Exception(
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); 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. /// Now only three parameters remain - date (or partitioning expression), primary_key, index_granularity.
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args[0].get())) if (!getIdentifierName(engine_args[0], date_column_name))
date_column_name = ast->name;
else
throw Exception( throw Exception(
"Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), "Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);

View File

@ -2,7 +2,6 @@
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
namespace DB namespace DB
@ -66,7 +65,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
PartitionCommand res; PartitionCommand res;
res.type = CLEAR_COLUMN; res.type = CLEAR_COLUMN;
res.partition = command_ast->partition; 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; res.column_name = column_name;
return res; 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 /// 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; source_fd = STDIN_FILENO;
else if (identifier->name == "stdout") else if (*opt_name == "stdout")
source_fd = STDOUT_FILENO; source_fd = STDOUT_FILENO;
else if (identifier->name == "stderr") else if (*opt_name == "stderr")
source_fd = STDERR_FILENO; source_fd = STDERR_FILENO;
else 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); ErrorCodes::UNKNOWN_IDENTIFIER);
} }
else if (const ASTLiteral * literal = typeid_cast<const ASTLiteral *>(engine_args[1].get())) else if (const ASTLiteral * literal = typeid_cast<const ASTLiteral *>(engine_args[1].get()))

View File

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

View File

@ -3,7 +3,6 @@
#include <Interpreters/Join.h> #include <Interpreters/Join.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Core/ColumnNumbers.h> #include <Core/ColumnNumbers.h>
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/NestedUtils.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...).", "Storage Join requires at least 3 parameters: Join(ANY|ALL, LEFT|INNER, keys...).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTIdentifier * strictness_id = typeid_cast<const ASTIdentifier *>(engine_args[0].get()); auto opt_strictness_id = getIdentifierName(engine_args[0]);
if (!strictness_id) if (!opt_strictness_id)
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS); 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; ASTTableJoin::Strictness strictness;
if (strictness_str == "any") if (strictness_str == "any")
strictness = ASTTableJoin::Strictness::Any; strictness = ASTTableJoin::Strictness::Any;
@ -101,11 +100,11 @@ void registerStorageJoin(StorageFactory & factory)
else else
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS); 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()); auto opt_kind_id = getIdentifierName(engine_args[1]);
if (!kind_id) if (!opt_kind_id)
throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS); 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; ASTTableJoin::Kind kind;
if (kind_str == "left") if (kind_str == "left")
kind = ASTTableJoin::Kind::Left; kind = ASTTableJoin::Kind::Left;
@ -122,11 +121,11 @@ void registerStorageJoin(StorageFactory & factory)
key_names.reserve(engine_args.size() - 2); key_names.reserve(engine_args.size() - 2);
for (size_t i = 2, size = engine_args.size(); i < size; ++i) for (size_t i = 2, size = engine_args.size(); i < size; ++i)
{ {
const ASTIdentifier * key = typeid_cast<const ASTIdentifier *>(engine_args[i].get()); auto opt_key = getIdentifierName(engine_args[i]);
if (!key) if (!opt_key)
throw Exception("Parameter №" + toString(i + 1) + " of storage Join don't look like column name.", ErrorCodes::BAD_ARGUMENTS); 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(); 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_sorting_key", std::make_shared<DataTypeUInt8>() },
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() }, { "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
{ "is_in_sampling_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; NamesAndTypesList columns;
ColumnDefaults column_defaults; ColumnDefaults column_defaults;
ColumnComments column_comments; ColumnComments column_comments;
ColumnCodecs column_codecs;
Names cols_required_for_partition_key; Names cols_required_for_partition_key;
Names cols_required_for_sorting_key; Names cols_required_for_sorting_key;
Names cols_required_for_primary_key; Names cols_required_for_primary_key;
@ -114,6 +116,7 @@ protected:
} }
columns = storage->getColumns().getAll(); columns = storage->getColumns().getAll();
column_codecs = storage->getColumns().codecs;
column_defaults = storage->getColumns().defaults; column_defaults = storage->getColumns().defaults;
column_comments = storage->getColumns().comments; column_comments = storage->getColumns().comments;
@ -219,6 +222,20 @@ protected:
res_columns[res_index++]->insert(find_in_vector(cols_required_for_sampling)); 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; ++rows_count;
} }
} }

View File

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

View File

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

View File

@ -1,3 +1,5 @@
#include "TableFunctionRemote.h"
#include <Storages/getStructureOfRemoteTable.h> #include <Storages/getStructureOfRemoteTable.h>
#include <Storages/StorageDistributed.h> #include <Storages/StorageDistributed.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
@ -8,10 +10,8 @@
#include <Interpreters/Cluster.h> #include <Interpreters/Cluster.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/parseRemoteDescription.h>
#include <TableFunctions/TableFunctionRemote.h>
#include <TableFunctions/TableFunctionFactory.h> #include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/parseRemoteDescription.h>
namespace DB namespace DB
@ -65,9 +65,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
} }
else else
{ {
if (auto ast_cluster = typeid_cast<const ASTIdentifier *>(args[arg_num].get())) if (!getIdentifierName(args[arg_num], cluster_name))
cluster_name = ast_cluster->name;
else
cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern"); cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern");
} }
++arg_num; ++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. /// 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. /// We need to mark them as the name of the database or table, because the default value is column.
for (auto & arg : args) for (auto ast : args)
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get())) setIdentifierSpecial(ast);
id->setSpecial();
ClusterPtr cluster; ClusterPtr cluster;
if (!cluster_name.empty()) if (!cluster_name.empty())

View File

@ -0,0 +1,33 @@
2018-01-01 1
2018-01-01 2
CODEC(ZSTD(1))
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(1), LZ4HC(0), 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(1), LZ4HC(0), LZ4, LZ4, NONE)
CODEC(NONE, LZ4, LZ4HC(0), ZSTD(1))
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;

Some files were not shown because too many files have changed in this diff Show More