Merge remote-tracking branch 'upstream/laplab-update_libunwind' into update_libunwind

This commit is contained in:
Nikita Lapkov 2019-04-21 09:22:46 +00:00
commit 1d121ca253
170 changed files with 4310 additions and 1186 deletions

View File

@ -28,12 +28,12 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
message (FATAL_ERROR "GCC version must be at least 7. For example, if GCC 7 is available under gcc-7, g++-7 names, do the following: export CC=gcc-7 CXX=g++-7; rm -rf CMakeCache.txt CMakeFiles; and re run cmake or ./release.")
endif ()
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
# Require at least clang 5
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 5)
message (FATAL_ERROR "Clang version must be at least 5.")
# Require at least clang 6
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 6)
message (FATAL_ERROR "Clang version must be at least 6.")
endif ()
else ()
message (WARNING "You are using an unsupported compiler. Compilation has only been tested with Clang 5+ and GCC 7+.")
message (WARNING "You are using an unsupported compiler. Compilation has only been tested with Clang 6+ and GCC 7+.")
endif ()
# Check that submodules are present only if source was downloaded with git
@ -99,10 +99,16 @@ if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64")
if (OS_LINUX AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0")
option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON)
endif ()
if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0")
message (WARNING "CMake version must be greater than 3.9.0 for production builds.")
endif ()
endif ()
if (GLIBC_COMPATIBILITY)
set (USE_INTERNAL_MEMCPY ON)
else ()
message (WARNING "Option GLIBC_COMPATIBILITY must be turned on for production builds.")
endif ()
string(REGEX MATCH "-?[0-9]+(.[0-9]+)?$" COMPILER_POSTFIX ${CMAKE_CXX_COMPILER})
@ -172,13 +178,17 @@ set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3
include (cmake/use_libcxx.cmake)
include (cmake/find_unwind.cmake)
if (USE_INTERNAL_UNWIND_LIBRARY)
option (USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING "Use internal unwind library for exception handling" 1)
endif ()
# Set standard, system and compiler libraries explicitly.
# This is intended for more control of what we are linking.
set (DEFAULT_LIBS "")
if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_LIBCXX))
if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING OR USE_LIBCXX))
# Note: this probably has no effect, but I'm not an expert in CMake.
set (CMAKE_C_IMPLICIT_LINK_LIBRARIES "")
set (CMAKE_CXX_IMPLICIT_LINK_LIBRARIES "")
@ -192,6 +202,9 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_LIBCXX))
execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE)
endif ()
string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC)
set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX")
# Add C++ libraries.
#
# This consist of:
@ -201,10 +214,18 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_LIBCXX))
# - compiler builtins (example: functions for implementation of __int128 operations);
#
# There are two variants of C++ library: libc++ (from LLVM compiler infrastructure) and libstdc++ (from GCC).
if (USE_LIBCXX)
set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lc++ -lc++abi -lgcc_eh ${BUILTINS_LIB_PATH} -Wl,-Bdynamic")
if (USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING)
# TODO: Allow to use non-static library as well.
set (EXCEPTION_HANDLING_LIBRARY "lib/libunwind${${CMAKE_POSTFIX_VARIABLE}}.a")
else ()
set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lstdc++ -lgcc_eh -lgcc ${BUILTINS_LIB_PATH} -Wl,-Bdynamic")
set (EXCEPTION_HANDLING_LIBRARY "-lgcc_eh")
endif ()
if (USE_LIBCXX)
set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lc++ -lc++abi ${EXCEPTION_HANDLING_LIBRARY} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic")
else ()
set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lstdc++ ${EXCEPTION_HANDLING_LIBRARY} -lgcc ${BUILTINS_LIB_PATH} -Wl,-Bdynamic")
endif ()
# Linking with GLIBC prevents portability of binaries to older systems.
@ -213,10 +234,7 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_LIBCXX))
if (GLIBC_COMPATIBILITY)
message (STATUS "Some symbols from glibc will be replaced for compatibility")
string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC)
set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX")
# FIXME: glibc-compatibility may be non-static in some builds!
# TODO: Allow to use non-static library as well.
set (DEFAULT_LIBS "${DEFAULT_LIBS} libs/libglibc-compatibility/libglibc-compatibility${${CMAKE_POSTFIX_VARIABLE}}.a")
endif ()
@ -332,7 +350,6 @@ include (libs/libcommon/cmake/find_gperftools.cmake)
include (libs/libcommon/cmake/find_jemalloc.cmake)
include (libs/libcommon/cmake/find_cctz.cmake)
include (libs/libmysqlxx/cmake/find_mysqlclient.cmake)
include (libs/libdaemon/cmake/find_unwind.cmake)
include (cmake/print_flags.cmake)
@ -343,30 +360,33 @@ add_subdirectory (dbms)
include (cmake/print_include_directories.cmake)
if (GLIBC_COMPATIBILITY)
if (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING)
# FIXME: actually glibc-compatibility should always be built first,
# because it's unconditionally linked via $DEFAULT_LIBS,
# and these looks like the first places that get linked.
function (add_glibc_compat target_name)
function (add_default_dependencies target_name)
if (TARGET ${target_name})
add_dependencies(${target_name} glibc-compatibility)
if (GLIBC_COMPATIBILITY)
add_dependencies(${target_name} glibc-compatibility)
endif ()
if (USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING)
add_dependencies(${target_name} unwind_static)
endif ()
endif ()
endfunction ()
add_glibc_compat(ltdl)
add_glibc_compat(zlibstatic)
add_glibc_compat(jemalloc)
add_glibc_compat(unwind)
add_glibc_compat(memcpy)
add_glibc_compat(Foundation)
add_glibc_compat(common)
add_glibc_compat(gtest)
add_glibc_compat(lz4)
add_glibc_compat(zstd)
add_glibc_compat(snappy)
add_glibc_compat(arrow)
add_glibc_compat(protoc)
add_glibc_compat(thrift_static)
add_glibc_compat(boost_regex_internal)
add_glibc_compat(unwind_static)
add_default_dependencies(ltdl)
add_default_dependencies(zlibstatic)
add_default_dependencies(jemalloc)
add_default_dependencies(memcpy)
add_default_dependencies(Foundation)
add_default_dependencies(common)
add_default_dependencies(gtest)
add_default_dependencies(lz4)
add_default_dependencies(zstd)
add_default_dependencies(snappy)
add_default_dependencies(arrow)
add_default_dependencies(protoc)
add_default_dependencies(thrift_static)
add_default_dependencies(boost_regex_internal)
endif ()

View File

@ -36,6 +36,8 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (ENABLE_DATA_SQLITE 0 CACHE BOOL "")
set (ENABLE_DATA_MYSQL 0 CACHE BOOL "")
set (ENABLE_DATA_POSTGRESQL 0 CACHE BOOL "")
set (ENABLE_ENCODINGS 0 CACHE BOOL "")
# new after 2.0.0:
set (POCO_ENABLE_ZIP 0 CACHE BOOL "")
set (POCO_ENABLE_PAGECOMPILER 0 CACHE BOOL "")

View File

@ -284,6 +284,7 @@ endif ()
if (USE_INTERNAL_BROTLI_LIBRARY)
add_subdirectory(brotli-cmake)
target_compile_definitions(brotli PRIVATE BROTLI_BUILD_PORTABLE=1)
endif ()
if (USE_INTERNAL_PROTOBUF_LIBRARY)

2
contrib/hyperscan vendored

@ -1 +1 @@
Subproject commit 05dab0efee80be405aad5f74721b692b6889b75e
Subproject commit 05b0f9064cca4bd55548dedb0a32ed9461146c1e

View File

@ -296,7 +296,7 @@ void HTTPHandler::processQuery(
/// The client can pass a HTTP header indicating supported compression method (gzip or deflate).
String http_response_compression_methods = request.get("Accept-Encoding", "");
bool client_supports_http_compression = false;
ZlibCompressionMethod http_response_compression_method {};
CompressionMethod http_response_compression_method {};
if (!http_response_compression_methods.empty())
{
@ -305,12 +305,17 @@ void HTTPHandler::processQuery(
if (std::string::npos != http_response_compression_methods.find("gzip"))
{
client_supports_http_compression = true;
http_response_compression_method = ZlibCompressionMethod::Gzip;
http_response_compression_method = CompressionMethod::Gzip;
}
else if (std::string::npos != http_response_compression_methods.find("deflate"))
{
client_supports_http_compression = true;
http_response_compression_method = ZlibCompressionMethod::Zlib;
http_response_compression_method = CompressionMethod::Zlib;
}
else if (http_response_compression_methods == "br")
{
client_supports_http_compression = true;
http_response_compression_method = CompressionMethod::Brotli;
}
}
@ -394,11 +399,11 @@ void HTTPHandler::processQuery(
{
if (http_request_compression_method_str == "gzip")
{
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, ZlibCompressionMethod::Gzip);
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Gzip);
}
else if (http_request_compression_method_str == "deflate")
{
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, ZlibCompressionMethod::Zlib);
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Zlib);
}
#if USE_BROTLI
else if (http_request_compression_method_str == "br")
@ -606,7 +611,7 @@ void HTTPHandler::processQuery(
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context,
[&response] (const String & content_type) { response.setContentType(content_type); },
[&response] (const String & current_query_id) { response.add("Query-Id", current_query_id); });
[&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); });
if (used_output.hasDelayed())
{

View File

@ -24,8 +24,7 @@ struct WithoutOverflowPolicy
static DataTypePtr promoteType(const DataTypePtr & data_type)
{
if (!data_type->canBePromoted())
throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return data_type->promoteNumericType();
}

View File

@ -271,7 +271,7 @@ private:
void initBlockInput();
void initBlockLogsInput();
void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
[[noreturn]] void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
};
}

View File

@ -422,6 +422,8 @@ namespace ErrorCodes
extern const int CANNOT_MPROTECT = 445;
extern const int FUNCTION_NOT_ALLOWED = 446;
extern const int HYPERSCAN_CANNOT_SCAN_TEXT = 447;
extern const int BROTLI_READ_FAILED = 448;
extern const int BROTLI_WRITE_FAILED = 449;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -14,7 +14,7 @@ namespace DB
class IDataTypeDummy : public DataTypeWithSimpleSerialization
{
private:
void throwNoSerialization() const
[[noreturn]] void throwNoSerialization() const
{
throw Exception("Serialization is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -77,7 +77,7 @@ namespace ProtobufColumnMatcher
namespace details
{
void throwNoCommonColumns();
[[noreturn]] void throwNoCommonColumns();
class ColumnNameMatcher
{

View File

@ -385,73 +385,61 @@ public:
bool readStringInto(PaddedPODArray<UInt8> &) override
{
cannotConvertType("String");
return false;
}
bool readInt8(Int8 &) override
{
cannotConvertType("Int8");
return false;
}
bool readUInt8(UInt8 &) override
{
cannotConvertType("UInt8");
return false;
}
bool readInt16(Int16 &) override
{
cannotConvertType("Int16");
return false;
}
bool readUInt16(UInt16 &) override
{
cannotConvertType("UInt16");
return false;
}
bool readInt32(Int32 &) override
{
cannotConvertType("Int32");
return false;
}
bool readUInt32(UInt32 &) override
{
cannotConvertType("UInt32");
return false;
}
bool readInt64(Int64 &) override
{
cannotConvertType("Int64");
return false;
}
bool readUInt64(UInt64 &) override
{
cannotConvertType("UInt64");
return false;
}
bool readUInt128(UInt128 &) override
{
cannotConvertType("UInt128");
return false;
}
bool readFloat32(Float32 &) override
{
cannotConvertType("Float32");
return false;
}
bool readFloat64(Float64 &) override
{
cannotConvertType("Float64");
return false;
}
void prepareEnumMapping8(const std::vector<std::pair<std::string, Int8>> &) override {}
@ -460,59 +448,50 @@ public:
bool readEnum8(Int8 &) override
{
cannotConvertType("Enum");
return false;
}
bool readEnum16(Int16 &) override
{
cannotConvertType("Enum");
return false;
}
bool readUUID(UUID &) override
{
cannotConvertType("UUID");
return false;
}
bool readDate(DayNum &) override
{
cannotConvertType("Date");
return false;
}
bool readDateTime(time_t &) override
{
cannotConvertType("DateTime");
return false;
}
bool readDecimal32(Decimal32 &, UInt32, UInt32) override
{
cannotConvertType("Decimal32");
return false;
}
bool readDecimal64(Decimal64 &, UInt32, UInt32) override
{
cannotConvertType("Decimal64");
return false;
}
bool readDecimal128(Decimal128 &, UInt32, UInt32) override
{
cannotConvertType("Decimal128");
return false;
}
bool readAggregateFunction(const AggregateFunctionPtr &, AggregateDataPtr, Arena &) override
{
cannotConvertType("AggregateFunction");
return false;
}
protected:
void cannotConvertType(const String & type_name)
[[noreturn]] void cannotConvertType(const String & type_name)
{
throw Exception(
String("Could not convert type '") + field->type_name() + "' from protobuf field '" + field->name() + "' to data type '"
@ -520,7 +499,7 @@ protected:
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value, const String & type_name)
[[noreturn]] void cannotConvertValue(const String & value, const String & type_name)
{
throw Exception(
"Could not convert value '" + value + "' from protobuf field '" + field->name() + "' to data type '" + type_name + "'",
@ -557,7 +536,6 @@ protected:
catch (...)
{
cannotConvertValue(StringRef(str.data(), str.size()).toString(), TypeName<To>::get());
__builtin_unreachable();
}
}

View File

@ -334,14 +334,14 @@ public:
virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) override { cannotConvertType("AggregateFunction"); }
protected:
void cannotConvertType(const String & type_name)
[[noreturn]] void cannotConvertType(const String & type_name)
{
throw Exception(
"Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value)
[[noreturn]] void cannotConvertValue(const String & value)
{
throw Exception(
"Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",

View File

@ -423,7 +423,7 @@ inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, Read
/** Throw exception with verbose message when string value is not parsed completely.
*/
void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
[[noreturn]] void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
enum class ConvertFromStringExceptionMode

View File

@ -520,7 +520,7 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
while (!sink.isEnd())
{
size_t row_num = array_source.rowNum();
bool has_size = !size_null_map || (size_null_map && (*size_null_map)[row_num]);
bool has_size = !size_null_map || (*size_null_map)[row_num];
if (has_size)
{

View File

@ -61,7 +61,7 @@ public:
{
auto array_size = col_num->getInt(i);
if (unlikely(array_size) < 0)
if (unlikely(array_size < 0))
throw Exception("Array size cannot be negative: while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
offset += array_size;

View File

@ -153,7 +153,7 @@ template <typename A, typename B>
struct NumIfImpl<A, B, NumberTraits::Error>
{
private:
static void throw_error()
[[noreturn]] static void throw_error()
{
throw Exception("Internal logic error: invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -0,0 +1,330 @@
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringOrArrayToT.h>
#include <cstring>
#ifdef __SSE4_1__
# include <emmintrin.h>
# include <smmintrin.h>
# include <tmmintrin.h>
#endif
namespace DB
{
/// inspired by https://github.com/cyb70289/utf8/
struct ValidUTF8Impl
{
/*
MIT License
Copyright (c) 2019 Yibo Cai
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.
*/
/*
* http://www.unicode.org/versions/Unicode6.0.0/ch03.pdf - page 94
*
* Table 3-7. Well-Formed UTF-8 Byte Sequences
*
* +--------------------+------------+-------------+------------+-------------+
* | Code Points | First Byte | Second Byte | Third Byte | Fourth Byte |
* +--------------------+------------+-------------+------------+-------------+
* | U+0000..U+007F | 00..7F | | | |
* +--------------------+------------+-------------+------------+-------------+
* | U+0080..U+07FF | C2..DF | 80..BF | | |
* +--------------------+------------+-------------+------------+-------------+
* | U+0800..U+0FFF | E0 | A0..BF | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+1000..U+CFFF | E1..EC | 80..BF | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+D000..U+D7FF | ED | 80..9F | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+E000..U+FFFF | EE..EF | 80..BF | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+10000..U+3FFFF | F0 | 90..BF | 80..BF | 80..BF |
* +--------------------+------------+-------------+------------+-------------+
* | U+40000..U+FFFFF | F1..F3 | 80..BF | 80..BF | 80..BF |
* +--------------------+------------+-------------+------------+-------------+
* | U+100000..U+10FFFF | F4 | 80..8F | 80..BF | 80..BF |
* +--------------------+------------+-------------+------------+-------------+
*/
static inline UInt8 isValidUTF8Naive(const UInt8 * data, UInt64 len)
{
while (len)
{
int bytes;
const UInt8 byte1 = data[0];
/* 00..7F */
if (byte1 <= 0x7F)
{
bytes = 1;
}
/* C2..DF, 80..BF */
else if (len >= 2 && byte1 >= 0xC2 && byte1 <= 0xDF && static_cast<Int8>(data[1]) <= static_cast<Int8>(0xBF))
{
bytes = 2;
}
else if (len >= 3)
{
const UInt8 byte2 = data[1];
bool byte2_ok = static_cast<Int8>(byte2) <= static_cast<Int8>(0xBF);
bool byte3_ok = static_cast<Int8>(data[2]) <= static_cast<Int8>(0xBF);
if (byte2_ok && byte3_ok &&
/* E0, A0..BF, 80..BF */
((byte1 == 0xE0 && byte2 >= 0xA0) ||
/* E1..EC, 80..BF, 80..BF */
(byte1 >= 0xE1 && byte1 <= 0xEC) ||
/* ED, 80..9F, 80..BF */
(byte1 == 0xED && byte2 <= 0x9F) ||
/* EE..EF, 80..BF, 80..BF */
(byte1 >= 0xEE && byte1 <= 0xEF)))
{
bytes = 3;
}
else if (len >= 4)
{
bool byte4_ok = static_cast<Int8>(data[3]) <= static_cast<Int8>(0xBF);
if (byte2_ok && byte3_ok && byte4_ok &&
/* F0, 90..BF, 80..BF, 80..BF */
((byte1 == 0xF0 && byte2 >= 0x90) ||
/* F1..F3, 80..BF, 80..BF, 80..BF */
(byte1 >= 0xF1 && byte1 <= 0xF3) ||
/* F4, 80..8F, 80..BF, 80..BF */
(byte1 == 0xF4 && byte2 <= 0x8F)))
{
bytes = 4;
}
else
{
return false;
}
}
else
{
return false;
}
}
else
{
return false;
}
len -= bytes;
data += bytes;
}
return true;
}
#ifndef __SSE4_1__
static inline UInt8 isValidUTF8(const UInt8 * data, UInt64 len) { return isValidUTF8Naive(data, len); }
#else
static inline UInt8 isValidUTF8(const UInt8 * data, UInt64 len)
{
/*
* Map high nibble of "First Byte" to legal character length minus 1
* 0x00 ~ 0xBF --> 0
* 0xC0 ~ 0xDF --> 1
* 0xE0 ~ 0xEF --> 2
* 0xF0 ~ 0xFF --> 3
*/
const __m128i first_len_tbl = _mm_setr_epi8(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 2, 3);
/* Map "First Byte" to 8-th item of range table (0xC2 ~ 0xF4) */
const __m128i first_range_tbl = _mm_setr_epi8(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 8, 8, 8, 8);
/*
* Range table, map range index to min and max values
*/
const __m128i range_min_tbl
= _mm_setr_epi8(0x00, 0x80, 0x80, 0x80, 0xA0, 0x80, 0x90, 0x80, 0xC2, 0x7F, 0x7F, 0x7F, 0x7F, 0x7F, 0x7F, 0x7F);
const __m128i range_max_tbl
= _mm_setr_epi8(0x7F, 0xBF, 0xBF, 0xBF, 0xBF, 0x9F, 0xBF, 0x8F, 0xF4, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80);
/*
* Tables for fast handling of four special First Bytes(E0,ED,F0,F4), after
* which the Second Byte are not 80~BF. It contains "range index adjustment".
* +------------+---------------+------------------+----------------+
* | First Byte | original range| range adjustment | adjusted range |
* +------------+---------------+------------------+----------------+
* | E0 | 2 | 2 | 4 |
* +------------+---------------+------------------+----------------+
* | ED | 2 | 3 | 5 |
* +------------+---------------+------------------+----------------+
* | F0 | 3 | 3 | 6 |
* +------------+---------------+------------------+----------------+
* | F4 | 4 | 4 | 8 |
* +------------+---------------+------------------+----------------+
*/
/* index1 -> E0, index14 -> ED */
const __m128i df_ee_tbl = _mm_setr_epi8(0, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3, 0);
/* index1 -> F0, index5 -> F4 */
const __m128i ef_fe_tbl = _mm_setr_epi8(0, 3, 0, 0, 0, 4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
__m128i prev_input = _mm_set1_epi8(0);
__m128i prev_first_len = _mm_set1_epi8(0);
__m128i error = _mm_set1_epi8(0);
auto check_packed = [&](__m128i input) noexcept
{
/* high_nibbles = input >> 4 */
const __m128i high_nibbles = _mm_and_si128(_mm_srli_epi16(input, 4), _mm_set1_epi8(0x0F));
/* first_len = legal character length minus 1 */
/* 0 for 00~7F, 1 for C0~DF, 2 for E0~EF, 3 for F0~FF */
/* first_len = first_len_tbl[high_nibbles] */
__m128i first_len = _mm_shuffle_epi8(first_len_tbl, high_nibbles);
/* First Byte: set range index to 8 for bytes within 0xC0 ~ 0xFF */
/* range = first_range_tbl[high_nibbles] */
__m128i range = _mm_shuffle_epi8(first_range_tbl, high_nibbles);
/* Second Byte: set range index to first_len */
/* 0 for 00~7F, 1 for C0~DF, 2 for E0~EF, 3 for F0~FF */
/* range |= (first_len, prev_first_len) << 1 byte */
range = _mm_or_si128(range, _mm_alignr_epi8(first_len, prev_first_len, 15));
/* Third Byte: set range index to saturate_sub(first_len, 1) */
/* 0 for 00~7F, 0 for C0~DF, 1 for E0~EF, 2 for F0~FF */
__m128i tmp1;
__m128i tmp2;
/* tmp1 = saturate_sub(first_len, 1) */
tmp1 = _mm_subs_epu8(first_len, _mm_set1_epi8(1));
/* tmp2 = saturate_sub(prev_first_len, 1) */
tmp2 = _mm_subs_epu8(prev_first_len, _mm_set1_epi8(1));
/* range |= (tmp1, tmp2) << 2 bytes */
range = _mm_or_si128(range, _mm_alignr_epi8(tmp1, tmp2, 14));
/* Fourth Byte: set range index to saturate_sub(first_len, 2) */
/* 0 for 00~7F, 0 for C0~DF, 0 for E0~EF, 1 for F0~FF */
/* tmp1 = saturate_sub(first_len, 2) */
tmp1 = _mm_subs_epu8(first_len, _mm_set1_epi8(2));
/* tmp2 = saturate_sub(prev_first_len, 2) */
tmp2 = _mm_subs_epu8(prev_first_len, _mm_set1_epi8(2));
/* range |= (tmp1, tmp2) << 3 bytes */
range = _mm_or_si128(range, _mm_alignr_epi8(tmp1, tmp2, 13));
/*
* Now we have below range indices caluclated
* Correct cases:
* - 8 for C0~FF
* - 3 for 1st byte after F0~FF
* - 2 for 1st byte after E0~EF or 2nd byte after F0~FF
* - 1 for 1st byte after C0~DF or 2nd byte after E0~EF or
* 3rd byte after F0~FF
* - 0 for others
* Error cases:
* 9,10,11 if non ascii First Byte overlaps
* E.g., F1 80 C2 90 --> 8 3 10 2, where 10 indicates error
*/
/* Adjust Second Byte range for special First Bytes(E0,ED,F0,F4) */
/* Overlaps lead to index 9~15, which are illegal in range table */
__m128i shift1, pos, range2;
/* shift1 = (input, prev_input) << 1 byte */
shift1 = _mm_alignr_epi8(input, prev_input, 15);
pos = _mm_sub_epi8(shift1, _mm_set1_epi8(0xEF));
/*
* shift1: | EF F0 ... FE | FF 00 ... ... DE | DF E0 ... EE |
* pos: | 0 1 15 | 16 17 239| 240 241 255|
* pos-240: | 0 0 0 | 0 0 0 | 0 1 15 |
* pos+112: | 112 113 127| >= 128 | >= 128 |
*/
tmp1 = _mm_subs_epu8(pos, _mm_set1_epi8(240));
range2 = _mm_shuffle_epi8(df_ee_tbl, tmp1);
tmp2 = _mm_adds_epu8(pos, _mm_set1_epi8(112));
range2 = _mm_add_epi8(range2, _mm_shuffle_epi8(ef_fe_tbl, tmp2));
range = _mm_add_epi8(range, range2);
/* Load min and max values per calculated range index */
__m128i minv = _mm_shuffle_epi8(range_min_tbl, range);
__m128i maxv = _mm_shuffle_epi8(range_max_tbl, range);
/* Check value range */
error = _mm_or_si128(error, _mm_cmplt_epi8(input, minv));
error = _mm_or_si128(error, _mm_cmpgt_epi8(input, maxv));
prev_input = input;
prev_first_len = first_len;
data += 16;
len -= 16;
};
while (len >= 16)
check_packed(_mm_loadu_si128(reinterpret_cast<const __m128i *>(data)));
/// 0 <= len <= 15 for now. Reading data from data - 1 because of right padding of 15 and left padding
/// Then zero some bytes from the unknown memory and check again.
alignas(16) char buf[32];
_mm_store_si128(reinterpret_cast<__m128i *>(buf), _mm_loadu_si128(reinterpret_cast<const __m128i *>(data - 1)));
memset(buf + len + 1, 0, 16);
check_packed(_mm_loadu_si128(reinterpret_cast<__m128i *>(buf + 1)));
/* Reduce error vector, error_reduced = 0xFFFF if error == 0 */
return _mm_movemask_epi8(_mm_cmpeq_epi8(error, _mm_set1_epi8(0))) == 0xFFFF;
}
#endif
static constexpr bool is_fixed_to_constant = false;
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt8> & res)
{
size_t size = offsets.size();
size_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = isValidUTF8(data.data() + prev_offset, offsets[i] - 1 - prev_offset);
prev_offset = offsets[i];
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/) {}
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
res[i] = isValidUTF8(data.data() + i * n, n);
}
static void array(const ColumnString::Offsets &, PaddedPODArray<UInt8> &)
{
throw Exception("Cannot apply function isValidUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct NameValidUTF8
{
static constexpr auto name = "isValidUTF8";
};
using FunctionValidUTF8 = FunctionStringOrArrayToT<ValidUTF8Impl, NameValidUTF8, UInt8>;
void registerFunctionValidUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionValidUTF8>();
}
}

View File

@ -9,6 +9,7 @@ void registerFunctionEmpty(FunctionFactory &);
void registerFunctionNotEmpty(FunctionFactory &);
void registerFunctionLength(FunctionFactory &);
void registerFunctionLengthUTF8(FunctionFactory &);
void registerFunctionValidUTF8(FunctionFactory &);
void registerFunctionLower(FunctionFactory &);
void registerFunctionUpper(FunctionFactory &);
void registerFunctionLowerUTF8(FunctionFactory &);
@ -36,6 +37,7 @@ void registerFunctionsString(FunctionFactory & factory)
registerFunctionNotEmpty(factory);
registerFunctionLength(factory);
registerFunctionLengthUTF8(factory);
registerFunctionValidUTF8(factory);
registerFunctionLower(factory);
registerFunctionUpper(factory);
registerFunctionLowerUTF8(factory);

View File

@ -7,6 +7,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BROTLI_READ_FAILED;
}
class BrotliReadBuffer::BrotliStateWrapper
{
public:
@ -29,7 +35,7 @@ public:
BrotliReadBuffer::BrotliReadBuffer(ReadBuffer &in_, size_t buf_size, char *existing_memory, size_t alignment)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
, in(in_)
, brotli(new BrotliStateWrapper())
, brotli(std::make_unique<BrotliStateWrapper>())
, in_available(0)
, in_data(nullptr)
, out_capacity(0)
@ -56,7 +62,7 @@ bool BrotliReadBuffer::nextImpl()
if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in.eof()))
{
throw Exception(std::string("brotli decode error"), ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
out_capacity = internal_buffer.size();
@ -76,13 +82,13 @@ bool BrotliReadBuffer::nextImpl()
}
else
{
throw Exception(std::string("brotli decode error"), ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
}
if (brotli->result == BROTLI_DECODER_RESULT_ERROR)
{
throw Exception(std::string("brotli decode error"), ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
return true;

View File

@ -34,5 +34,6 @@ private:
bool eof;
};
}

View File

@ -0,0 +1,126 @@
#include <Common/config.h>
#if USE_BROTLI
#include <IO/BrotliWriteBuffer.h>
#include <brotli/encode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BROTLI_WRITE_FAILED;
}
class BrotliWriteBuffer::BrotliStateWrapper
{
public:
BrotliStateWrapper()
: state(BrotliEncoderCreateInstance(nullptr, nullptr, nullptr))
{
}
~BrotliStateWrapper()
{
BrotliEncoderDestroyInstance(state);
}
public:
BrotliEncoderState * state;
};
BrotliWriteBuffer::BrotliWriteBuffer(WriteBuffer & out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
, brotli(std::make_unique<BrotliStateWrapper>())
, in_available(0)
, in_data(nullptr)
, out_capacity(0)
, out_data(nullptr)
, out(out_)
{
BrotliEncoderSetParameter(brotli->state, BROTLI_PARAM_QUALITY, static_cast<uint32_t>(compression_level));
// Set LZ77 window size. According to brotli sources default value is 24 (c/tools/brotli.c:81)
BrotliEncoderSetParameter(brotli->state, BROTLI_PARAM_LGWIN, 24);
}
BrotliWriteBuffer::~BrotliWriteBuffer()
{
finish();
}
void BrotliWriteBuffer::nextImpl()
{
if (!offset())
{
return;
}
in_data = reinterpret_cast<unsigned char *>(working_buffer.begin());
in_available = offset();
do
{
out.nextIfAtEnd();
out_data = reinterpret_cast<unsigned char *>(out.position());
out_capacity = out.buffer().end() - out.position();
int result = BrotliEncoderCompressStream(
brotli->state,
in_available ? BROTLI_OPERATION_PROCESS : BROTLI_OPERATION_FINISH,
&in_available,
&in_data,
&out_capacity,
&out_data,
nullptr);
out.position() = out.buffer().end() - out_capacity;
if (result == 0)
{
throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
}
}
while (in_available > 0 || out_capacity == 0);
}
void BrotliWriteBuffer::finish()
{
if (finished)
return;
next();
while (true)
{
out.nextIfAtEnd();
out_data = reinterpret_cast<unsigned char *>(out.position());
out_capacity = out.buffer().end() - out.position();
int result = BrotliEncoderCompressStream(
brotli->state,
BROTLI_OPERATION_FINISH,
&in_available,
&in_data,
&out_capacity,
&out_data,
nullptr);
out.position() = out.buffer().end() - out_capacity;
if (BrotliEncoderIsFinished(brotli->state))
{
finished = true;
return;
}
if (result == 0)
{
throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
}
}
}
}
#endif

View File

@ -0,0 +1,40 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
class BrotliWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{
public:
BrotliWriteBuffer(
WriteBuffer & out_,
int compression_level,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
~BrotliWriteBuffer() override;
void finish();
private:
void nextImpl() override;
class BrotliStateWrapper;
std::unique_ptr<BrotliStateWrapper> brotli;
size_t in_available;
const uint8_t * in_data;
size_t out_capacity;
uint8_t * out_data;
WriteBuffer & out;
bool finished = false;
};
}

View File

@ -3,7 +3,7 @@
namespace DB
{
enum class ZlibCompressionMethod
enum class CompressionMethod
{
/// DEFLATE compression with gzip header and CRC32 checksum.
/// This option corresponds to files produced by gzip(1) or HTTP Content-Encoding: gzip.
@ -11,6 +11,7 @@ enum class ZlibCompressionMethod
/// DEFLATE compression with zlib header and Adler32 checksum.
/// This option corresponds to HTTP Content-Encoding: deflate.
Zlib,
Brotli,
};
}

View File

@ -179,7 +179,7 @@ private:
*/
virtual bool nextImpl() { return false; }
void throwReadAfterEOF()
[[noreturn]] void throwReadAfterEOF()
{
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}

View File

@ -164,7 +164,7 @@ void readVectorBinary(std::vector<T> & v, ReadBuffer & buf, size_t MAX_VECTOR_SI
void assertString(const char * s, ReadBuffer & buf);
void assertEOF(ReadBuffer & buf);
void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
[[noreturn]] void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
inline void assertChar(char symbol, ReadBuffer & buf)
{

View File

@ -113,7 +113,7 @@ readVarUInt(T & x, ReadBuffer & istr)
}
inline void throwReadAfterEOF()
[[noreturn]] inline void throwReadAfterEOF()
{
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}

View File

@ -76,34 +76,47 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
{
if (compress)
{
if (compression_method == ZlibCompressionMethod::Gzip)
if (compression_method == CompressionMethod::Gzip)
{
#if defined(POCO_CLICKHOUSE_PATCH)
*response_header_ostr << "Content-Encoding: gzip\r\n";
#else
response.set("Content-Encoding", "gzip");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else if (compression_method == ZlibCompressionMethod::Zlib)
else if (compression_method == CompressionMethod::Zlib)
{
#if defined(POCO_CLICKHOUSE_PATCH)
*response_header_ostr << "Content-Encoding: deflate\r\n";
#else
response.set("Content-Encoding", "deflate");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else if (compression_method == CompressionMethod::Brotli)
{
#if defined(POCO_CLICKHOUSE_PATCH)
*response_header_ostr << "Content-Encoding: br\r\n";
#else
response.set("Content-Encoding", "br");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin());
out = &*brotli_buf;
}
else
throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse",
ErrorCodes::LOGICAL_ERROR);
/// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy.
#if !defined(POCO_CLICKHOUSE_PATCH)
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else
{
@ -133,7 +146,7 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse(
Poco::Net::HTTPServerResponse & response_,
unsigned keep_alive_timeout_,
bool compress_,
ZlibCompressionMethod compression_method_,
CompressionMethod compression_method_,
size_t size)
: BufferWithOwnMemory<WriteBuffer>(size)
, request(request_)

View File

@ -9,6 +9,7 @@
#include <IO/BufferWithOwnMemory.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/ZlibDeflatingWriteBuffer.h>
#include <IO/BrotliWriteBuffer.h>
#include <IO/HTTPCommon.h>
#include <IO/Progress.h>
#include <Common/NetException.h>
@ -49,7 +50,7 @@ private:
bool add_cors_header = false;
unsigned keep_alive_timeout = 0;
bool compress = false;
ZlibCompressionMethod compression_method;
CompressionMethod compression_method;
int compression_level = Z_DEFAULT_COMPRESSION;
std::ostream * response_body_ostr = nullptr;
@ -60,6 +61,7 @@ private:
std::optional<WriteBufferFromOStream> out_raw;
std::optional<ZlibDeflatingWriteBuffer> deflating_buf;
std::optional<BrotliWriteBuffer> brotli_buf;
WriteBuffer * out = nullptr; /// Uncompressed HTTP body is written to this buffer. Points to out_raw or possibly to deflating_buf.
@ -89,7 +91,7 @@ public:
Poco::Net::HTTPServerResponse & response_,
unsigned keep_alive_timeout_,
bool compress_ = false, /// If true - set Content-Encoding header and compress the result.
ZlibCompressionMethod compression_method_ = ZlibCompressionMethod::Gzip,
CompressionMethod compression_method_ = CompressionMethod::Gzip,
size_t size = DBMS_DEFAULT_BUFFER_SIZE);
/// Writes progess in repeating HTTP headers.

View File

@ -6,7 +6,7 @@ namespace DB
ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
WriteBuffer & out_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
int compression_level,
size_t buf_size,
char * existing_memory,
@ -23,7 +23,7 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
zstr.avail_out = 0;
int window_bits = 15;
if (compression_method == ZlibCompressionMethod::Gzip)
if (compression_method == CompressionMethod::Gzip)
{
window_bits += 16;
}

View File

@ -2,7 +2,7 @@
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ZlibCompressionMethod.h>
#include <IO/CompressionMethod.h>
#include <zlib.h>
@ -21,7 +21,7 @@ class ZlibDeflatingWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
public:
ZlibDeflatingWriteBuffer(
WriteBuffer & out_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
int compression_level,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,

View File

@ -6,7 +6,7 @@ namespace DB
ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
ReadBuffer & in_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
size_t buf_size,
char * existing_memory,
size_t alignment)
@ -23,7 +23,7 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
zstr.avail_out = 0;
int window_bits = 15;
if (compression_method == ZlibCompressionMethod::Gzip)
if (compression_method == CompressionMethod::Gzip)
{
window_bits += 16;
}

View File

@ -2,7 +2,7 @@
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ZlibCompressionMethod.h>
#include <IO/CompressionMethod.h>
#include <zlib.h>
@ -22,7 +22,7 @@ class ZlibInflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
public:
ZlibInflatingReadBuffer(
ReadBuffer & in_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);

View File

@ -23,7 +23,7 @@ try
{
DB::WriteBufferFromFile buf("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC);
DB::ZlibDeflatingWriteBuffer deflating_buf(buf, DB::ZlibCompressionMethod::Gzip, /* compression_level = */ 3);
DB::ZlibDeflatingWriteBuffer deflating_buf(buf, DB::CompressionMethod::Gzip, /* compression_level = */ 3);
stopwatch.restart();
for (size_t i = 0; i < n; ++i)
@ -41,7 +41,7 @@ try
{
DB::ReadBufferFromFile buf("test_zlib_buffers.gz");
DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::ZlibCompressionMethod::Gzip);
DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::CompressionMethod::Gzip);
stopwatch.restart();
for (size_t i = 0; i < n; ++i)

View File

@ -71,8 +71,8 @@ private:
void visit(ASTSelectQuery & select, ASTPtr &) const
{
if (select.tables)
tryVisit<ASTTablesInSelectQuery>(select.tables);
if (select.tables())
tryVisit<ASTTablesInSelectQuery>(select.refTables());
visitChildren(select);
}

View File

@ -210,10 +210,10 @@ using CheckExpressionVisitor = InDepthNodeVisitor<CheckExpressionMatcher, true>;
bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables, size_t & num_comma)
{
if (!select.tables)
if (!select.tables())
return false;
const auto * tables = select.tables->as<ASTTablesInSelectQuery>();
const auto * tables = select.tables()->as<ASTTablesInSelectQuery>();
if (!tables)
return false;
@ -222,20 +222,38 @@ bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables
return false;
joined_tables.reserve(num_tables);
size_t num_array_join = 0;
size_t num_using = 0;
for (auto & child : tables->children)
{
joined_tables.emplace_back(JoinedTable(child));
JoinedTable & t = joined_tables.back();
if (t.array_join)
return false;
{
++num_array_join;
continue;
}
if (num_tables > 2 && t.has_using)
throw Exception("Multiple CROSS/COMMA JOIN do not support USING", ErrorCodes::NOT_IMPLEMENTED);
if (t.has_using)
{
++num_using;
continue;
}
if (auto * join = t.join)
if (join->kind == ASTTableJoin::Kind::Comma)
++num_comma;
}
if (num_using && (num_tables - num_array_join) > 2)
throw Exception("Multiple CROSS/COMMA JOIN do not support USING", ErrorCodes::NOT_IMPLEMENTED);
if (num_comma && (num_comma != (joined_tables.size() - 1)))
throw Exception("Mix of COMMA and other JOINS is not supported", ErrorCodes::NOT_IMPLEMENTED);
if (num_array_join || num_using)
return false;
return true;
}
@ -259,20 +277,17 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
if (num_comma)
{
if (num_comma != (joined_tables.size() - 1))
throw Exception("Mix of COMMA and other JOINS is not supported", ErrorCodes::NOT_IMPLEMENTED);
for (auto & table : joined_tables)
table.rewriteCommaToCross();
}
/// CROSS to INNER
if (!select.where_expression)
if (!select.where())
return;
CheckExpressionVisitor::Data visitor_data{joined_tables};
CheckExpressionVisitor(visitor_data).visit(select.where_expression);
CheckExpressionVisitor(visitor_data).visit(select.refWhere());
if (visitor_data.complex())
return;

View File

@ -71,12 +71,12 @@ String DatabaseAndTableWithAlias::getQualifiedNamePrefix(bool with_dot) const
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery & select_query)
{
if (!select_query.tables)
if (!select_query.tables())
return {};
std::vector<const ASTTableExpression *> tables_expression;
for (const auto & child : select_query.tables->children)
for (const auto & child : select_query.tables()->children)
{
const auto * tables_element = child->as<ASTTablesInSelectQueryElement>();
@ -89,10 +89,10 @@ std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelec
static const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, size_t table_number)
{
if (!select.tables)
if (!select.tables())
return {};
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
const auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
if (tables_in_select_query.children.size() <= table_number)
return {};
@ -151,7 +151,7 @@ std::vector<TableWithColumnNames> getDatabaseAndTablesWithColumnNames(const ASTS
{
std::vector<TableWithColumnNames> tables_with_columns;
if (select_query.tables && !select_query.tables->children.empty())
if (select_query.tables() && !select_query.tables()->children.empty())
{
String current_database = context.getCurrentDatabase();

View File

@ -131,7 +131,7 @@ void ExpressionAnalyzer::analyzeAggregation()
auto * select_query = query->as<ASTSelectQuery>();
if (select_query && (select_query->group_expression_list || select_query->having_expression))
if (select_query && (select_query->groupBy() || select_query->having()))
has_aggregation = true;
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(source_columns, context);
@ -168,10 +168,10 @@ void ExpressionAnalyzer::analyzeAggregation()
assertSelect();
/// Find out aggregation keys.
if (select_query->group_expression_list)
if (select_query->groupBy())
{
NameSet unique_keys;
ASTs & group_asts = select_query->group_expression_list->children;
ASTs & group_asts = select_query->groupBy()->children;
for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i)
{
ssize_t size = group_asts.size();
@ -216,8 +216,8 @@ void ExpressionAnalyzer::analyzeAggregation()
if (group_asts.empty())
{
select_query->group_expression_list = nullptr;
has_aggregation = select_query->having_expression || aggregate_descriptions.size();
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {});
has_aggregation = select_query->having() || aggregate_descriptions.size();
}
}
@ -255,10 +255,10 @@ void ExpressionAnalyzer::makeSetsForIndex()
if (storage && select_query && storage->supportsIndexForIn())
{
if (select_query->where_expression)
makeSetsForIndexImpl(select_query->where_expression);
if (select_query->prewhere_expression)
makeSetsForIndexImpl(select_query->prewhere_expression);
if (select_query->where())
makeSetsForIndexImpl(select_query->where());
if (select_query->prewhere())
makeSetsForIndexImpl(select_query->prewhere());
}
}
@ -371,7 +371,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
const auto * select_query = query->as<ASTSelectQuery>();
/// There can not be aggregate functions inside the WHERE and PREWHERE.
if (select_query && (ast.get() == select_query->where_expression.get() || ast.get() == select_query->prewhere_expression.get()))
if (select_query && (ast.get() == select_query->where().get() || ast.get() == select_query->prewhere().get()))
{
assertNoAggregates(ast, "in WHERE or PREWHERE");
return;
@ -637,20 +637,20 @@ bool ExpressionAnalyzer::appendPrewhere(
assertSelect();
if (!select_query->prewhere_expression)
if (!select_query->prewhere())
return false;
initChain(chain, source_columns);
auto & step = chain.getLastStep();
getRootActions(select_query->prewhere_expression, only_types, step.actions);
String prewhere_column_name = select_query->prewhere_expression->getColumnName();
getRootActions(select_query->prewhere(), only_types, step.actions);
String prewhere_column_name = select_query->prewhere()->getColumnName();
step.required_output.push_back(prewhere_column_name);
step.can_remove_required_output.push_back(true);
{
/// Remove unused source_columns from prewhere actions.
auto tmp_actions = std::make_shared<ExpressionActions>(source_columns, context);
getRootActions(select_query->prewhere_expression, only_types, tmp_actions);
getRootActions(select_query->prewhere(), only_types, tmp_actions);
tmp_actions->finalize({prewhere_column_name});
auto required_columns = tmp_actions->getRequiredColumns();
NameSet required_source_columns(required_columns.begin(), required_columns.end());
@ -712,16 +712,16 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t
assertSelect();
if (!select_query->where_expression)
if (!select_query->where())
return false;
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
step.required_output.push_back(select_query->where_expression->getColumnName());
step.required_output.push_back(select_query->where()->getColumnName());
step.can_remove_required_output = {true};
getRootActions(select_query->where_expression, only_types, step.actions);
getRootActions(select_query->where(), only_types, step.actions);
return true;
}
@ -732,13 +732,13 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only
assertAggregation();
if (!select_query->group_expression_list)
if (!select_query->groupBy())
return false;
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
ASTs asts = select_query->group_expression_list->children;
ASTs asts = select_query->groupBy()->children;
for (size_t i = 0; i < asts.size(); ++i)
{
step.required_output.push_back(asts[i]->getColumnName());
@ -765,13 +765,13 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai
}
}
getActionsBeforeAggregation(select_query->select_expression_list, step.actions, only_types);
getActionsBeforeAggregation(select_query->select(), step.actions, only_types);
if (select_query->having_expression)
getActionsBeforeAggregation(select_query->having_expression, step.actions, only_types);
if (select_query->having())
getActionsBeforeAggregation(select_query->having(), step.actions, only_types);
if (select_query->order_expression_list)
getActionsBeforeAggregation(select_query->order_expression_list, step.actions, only_types);
if (select_query->orderBy())
getActionsBeforeAggregation(select_query->orderBy(), step.actions, only_types);
}
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
@ -780,14 +780,14 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_
assertAggregation();
if (!select_query->having_expression)
if (!select_query->having())
return false;
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
step.required_output.push_back(select_query->having_expression->getColumnName());
getRootActions(select_query->having_expression, only_types, step.actions);
step.required_output.push_back(select_query->having()->getColumnName());
getRootActions(select_query->having(), only_types, step.actions);
return true;
}
@ -801,9 +801,9 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->select_expression_list, only_types, step.actions);
getRootActions(select_query->select(), only_types, step.actions);
for (const auto & child : select_query->select_expression_list->children)
for (const auto & child : select_query->select()->children)
step.required_output.push_back(child->getColumnName());
}
@ -813,18 +813,17 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
assertSelect();
if (!select_query->order_expression_list)
if (!select_query->orderBy())
return false;
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->order_expression_list, only_types, step.actions);
getRootActions(select_query->orderBy(), only_types, step.actions);
ASTs asts = select_query->order_expression_list->children;
for (size_t i = 0; i < asts.size(); ++i)
for (auto & child : select_query->orderBy()->children)
{
const auto * ast = asts[i]->as<ASTOrderByElement>();
const auto * ast = child->as<ASTOrderByElement>();
if (!ast || ast->children.size() < 1)
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
ASTPtr order_expression = ast->children.at(0);
@ -840,15 +839,15 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only
assertSelect();
if (!select_query->limit_by_expression_list)
if (!select_query->limitBy())
return false;
initChain(chain, aggregated_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->limit_by_expression_list, only_types, step.actions);
getRootActions(select_query->limitBy(), only_types, step.actions);
for (const auto & child : select_query->limit_by_expression_list->children)
for (const auto & child : select_query->limitBy()->children)
step.required_output.push_back(child->getColumnName());
return true;
@ -865,7 +864,7 @@ void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) con
NamesWithAliases result_columns;
ASTs asts = select_query->select_expression_list->children;
ASTs asts = select_query->select()->children;
for (size_t i = 0; i < asts.size(); ++i)
{
String result_name = asts[i]->getAliasOrColumnName();

View File

@ -46,48 +46,36 @@ public:
has_global_subqueries(has_global_subqueries_)
{}
void addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression)
void addExternalStorage(ASTPtr & ast, bool set_alias = false)
{
/// With nondistributed queries, creating temporary tables does not make sense.
if (!is_remote)
return;
ASTPtr subquery;
ASTPtr table_name;
ASTPtr subquery_or_table_name;
bool is_table = false;
ASTPtr subquery_or_table_name = ast; /// ASTIdentifier | ASTSubquery | ASTTableExpression
if (subquery_or_table_name_or_table_expression->as<ASTIdentifier>())
{
table_name = subquery_or_table_name_or_table_expression;
subquery_or_table_name = table_name;
}
else if (const auto * ast_table_expr = subquery_or_table_name_or_table_expression->as<ASTTableExpression>())
if (const auto * ast_table_expr = ast->as<ASTTableExpression>())
{
subquery_or_table_name = ast_table_expr->subquery;
if (ast_table_expr->database_and_table_name)
{
table_name = ast_table_expr->database_and_table_name;
subquery_or_table_name = table_name;
}
else if (ast_table_expr->subquery)
{
subquery = ast_table_expr->subquery;
subquery_or_table_name = subquery;
subquery_or_table_name = ast_table_expr->database_and_table_name;
is_table = true;
}
}
else if (subquery_or_table_name_or_table_expression->as<ASTSubquery>())
{
subquery = subquery_or_table_name_or_table_expression;
subquery_or_table_name = subquery;
}
else if (ast->as<ASTIdentifier>())
is_table = true;
if (!subquery_or_table_name)
throw Exception("Logical error: unknown AST element passed to ExpressionAnalyzer::addExternalStorage method",
ErrorCodes::LOGICAL_ERROR);
if (table_name)
if (is_table)
{
/// If this is already an external table, you do not need to add anything. Just remember its presence.
if (external_tables.end() != external_tables.find(*getIdentifierName(table_name)))
if (external_tables.end() != external_tables.find(*getIdentifierName(subquery_or_table_name)))
return;
}
@ -114,8 +102,16 @@ public:
*/
auto database_and_table_name = createTableIdentifier("", external_table_name);
if (set_alias)
{
String alias = subquery_or_table_name->tryGetAlias();
if (auto * table_name = subquery_or_table_name->as<ASTIdentifier>())
if (alias.empty())
alias = table_name->shortName();
database_and_table_name->setAlias(alias);
}
if (auto * ast_table_expr = subquery_or_table_name_or_table_expression->as<ASTTableExpression>())
if (auto * ast_table_expr = ast->as<ASTTableExpression>())
{
ast_table_expr->subquery.reset();
ast_table_expr->database_and_table_name = database_and_table_name;
@ -124,7 +120,7 @@ public:
ast_table_expr->children.emplace_back(database_and_table_name);
}
else
subquery_or_table_name_or_table_expression = database_and_table_name;
ast = database_and_table_name;
external_tables[external_table_name] = external_storage;
subqueries_for_sets[external_table_name].source = interpreter->execute().in;
@ -170,7 +166,7 @@ private:
{
if (table_elem.table_join && table_elem.table_join->as<ASTTableJoin &>().locality == ASTTableJoin::Locality::Global)
{
data.addExternalStorage(table_elem.table_expression);
data.addExternalStorage(table_elem.table_expression, true);
data.has_global_subqueries = true;
}
}

View File

@ -100,10 +100,10 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
if (distributed_product_mode == DistributedProductMode::ALLOW)
return;
if (!query->tables)
if (!query->tables())
return;
const auto & tables_in_select_query = query->tables->as<ASTTablesInSelectQuery &>();
const auto & tables_in_select_query = query->tables()->as<ASTTablesInSelectQuery &>();
if (tables_in_select_query.children.empty())
return;

View File

@ -90,9 +90,8 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr &
ASTPtr query_ast = std::make_shared<ASTSelectQuery>();
auto * select_ast = query_ast->as<ASTSelectQuery>();
auto expr_list = std::make_shared<ASTExpressionList>();
select_ast->children.push_back(expr_list);
select_ast->select_expression_list = select_ast->children.back();
select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
auto expr_list = select_ast->select();
auto parseExpression = [] (const String & expr)
{
@ -107,11 +106,10 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr &
for (const auto & column_str : prerequisite_columns)
expr_list->children.push_back(parseExpression(column_str));
auto tables = std::make_shared<ASTTablesInSelectQuery>();
select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
auto tables = select_ast->tables();
auto tables_elem = std::make_shared<ASTTablesInSelectQueryElement>();
auto table_expr = std::make_shared<ASTTableExpression>();
select_ast->children.push_back(tables);
select_ast->tables = select_ast->children.back();
tables->children.push_back(tables_elem);
tables_elem->table_expression = table_expr;
tables_elem->children.push_back(table_expr);
@ -270,7 +268,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (query.final() && (input || !storage || !storage->supportsFinal()))
throw Exception((!input && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL);
if (query.prewhere_expression && (input || !storage || !storage->supportsPrewhere()))
if (query.prewhere() && (input || !storage || !storage->supportsPrewhere()))
throw Exception((!input && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE);
/// Save the new temporary tables in the query context
@ -465,7 +463,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage
has_prewhere = true;
res.prewhere_info = std::make_shared<PrewhereInfo>(
chain.steps.front().actions, query.prewhere_expression->getColumnName());
chain.steps.front().actions, query.prewhere()->getColumnName());
chain.addStep();
}
@ -591,7 +589,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
query_info.sets = query_analyzer->getPreparedSets();
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
if (settings.optimize_move_to_prewhere && query.where_expression && !query.prewhere_expression && !query.final())
if (settings.optimize_move_to_prewhere && query.where() && !query.prewhere() && !query.final())
MergeTreeWhereOptimizer{query_info, context, merge_tree.getData(), query_analyzer->getRequiredSourceColumns(), log};
};
@ -707,7 +705,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
if (expressions.has_order_by)
executeOrder(pipeline);
if (expressions.has_order_by && query.limit_length)
if (expressions.has_order_by && query.limitLength())
executeDistinct(pipeline, false, expressions.selected_columns);
if (expressions.has_limit_by)
@ -716,7 +714,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
executeLimitBy(pipeline);
}
if (query.limit_length)
if (query.limitLength())
executePreLimit(pipeline);
}
@ -805,14 +803,14 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
/** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT,
* limiting the number of rows in each up to `offset + limit`.
*/
if (query.limit_length && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.has_limit_by && !settings.extremes)
if (query.limitLength() && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.has_limit_by && !settings.extremes)
{
executePreLimit(pipeline);
}
if (need_second_distinct_pass
|| query.limit_length
|| query.limit_by_expression_list
|| query.limitLength()
|| query.limitBy()
|| pipeline.stream_with_non_joined_data)
{
need_merge_streams = true;
@ -869,11 +867,11 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
UInt64 length = 0;
UInt64 offset = 0;
if (query.limit_length)
if (query.limitLength())
{
length = getLimitUIntValue(query.limit_length, context);
if (query.limit_offset)
offset = getLimitUIntValue(query.limit_offset, context);
length = getLimitUIntValue(query.limitLength(), context);
if (query.limitOffset())
offset = getLimitUIntValue(query.limitOffset(), context);
}
return {length, offset};
@ -882,7 +880,7 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context)
{
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY.
if (!query.distinct && !query.limit_by_expression_list)
if (!query.distinct && !query.limitBy())
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
return limit_length + limit_offset;
@ -1090,13 +1088,13 @@ void InterpreterSelectQuery::executeFetchColumns(
* and also set the number of threads to 1.
*/
if (!query.distinct
&& !query.prewhere_expression
&& !query.where_expression
&& !query.group_expression_list
&& !query.having_expression
&& !query.order_expression_list
&& !query.limit_by_expression_list
&& query.limit_length
&& !query.prewhere()
&& !query.where()
&& !query.groupBy()
&& !query.having()
&& !query.orderBy()
&& !query.limitBy()
&& query.limitLength()
&& !query_analyzer->hasAggregation()
&& limit_length + limit_offset < max_block_size)
{
@ -1224,7 +1222,7 @@ void InterpreterSelectQuery::executeWhere(Pipeline & pipeline, const ExpressionA
{
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<FilterBlockInputStream>(stream, expression, getSelectQuery().where_expression->getColumnName(), remove_fiter);
stream = std::make_shared<FilterBlockInputStream>(stream, expression, getSelectQuery().where()->getColumnName(), remove_fiter);
});
}
@ -1352,7 +1350,7 @@ void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const Expression
{
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<FilterBlockInputStream>(stream, expression, getSelectQuery().having_expression->getColumnName());
stream = std::make_shared<FilterBlockInputStream>(stream, expression, getSelectQuery().having()->getColumnName());
});
}
@ -1367,7 +1365,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
pipeline.firstStream(),
overflow_row,
expression,
has_having ? getSelectQuery().having_expression->getColumnName() : "",
has_having ? getSelectQuery().having()->getColumnName() : "",
settings.totals_mode,
settings.totals_auto_threshold,
final);
@ -1416,8 +1414,8 @@ void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const Expres
static SortDescription getSortDescription(const ASTSelectQuery & query)
{
SortDescription order_descr;
order_descr.reserve(query.order_expression_list->children.size());
for (const auto & elem : query.order_expression_list->children)
order_descr.reserve(query.orderBy()->children.size());
for (const auto & elem : query.orderBy()->children)
{
String name = elem->children.front()->getColumnName();
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
@ -1513,7 +1511,7 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or
UInt64 limit_for_distinct = 0;
/// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows.
if (!query.order_expression_list || !before_order)
if (!query.orderBy() || !before_order)
limit_for_distinct = limit_length + limit_offset;
pipeline.transform([&](auto & stream)
@ -1549,7 +1547,7 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
{
auto & query = getSelectQuery();
/// If there is LIMIT
if (query.limit_length)
if (query.limitLength())
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
pipeline.transform([&, limit = limit_length + limit_offset](auto & stream)
@ -1563,14 +1561,14 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
{
auto & query = getSelectQuery();
if (!query.limit_by_value || !query.limit_by_expression_list)
if (!query.limitByValue() || !query.limitBy())
return;
Names columns;
for (const auto & elem : query.limit_by_expression_list->children)
for (const auto & elem : query.limitBy()->children)
columns.emplace_back(elem->getColumnName());
UInt64 value = getLimitUIntValue(query.limit_by_value, context);
UInt64 value = getLimitUIntValue(query.limitByValue(), context);
pipeline.transform([&](auto & stream)
{
@ -1607,7 +1605,7 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
{
auto & query = getSelectQuery();
/// If there is LIMIT
if (query.limit_length)
if (query.limitLength())
{
/** Rare case:
* if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels,
@ -1620,7 +1618,7 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
*/
bool always_read_till_end = false;
if (query.group_by_with_totals && !query.order_expression_list)
if (query.group_by_with_totals && !query.orderBy())
always_read_till_end = true;
if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query))
@ -1684,8 +1682,8 @@ void InterpreterSelectQuery::ignoreWithTotals()
void InterpreterSelectQuery::initSettings()
{
auto & query = getSelectQuery();
if (query.settings)
InterpreterSetQuery(query.settings, context).executeForCurrentContext();
if (query.settings())
InterpreterSetQuery(query.settings(), context).executeForCurrentContext();
}
}

View File

@ -91,19 +91,11 @@ private:
if (data.table_columns.empty())
return;
Visitor(data).visit(node.select_expression_list);
Visitor(data).visit(node.refSelect());
if (!data.new_select_expression_list)
return;
size_t pos = 0;
for (; pos < node.children.size(); ++pos)
if (node.children[pos].get() == node.select_expression_list.get())
break;
if (pos == node.children.size())
throw Exception("No select expressions list in select", ErrorCodes::NOT_IMPLEMENTED);
node.select_expression_list = data.new_select_expression_list;
node.children[pos] = node.select_expression_list;
node.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.new_select_expression_list));
}
static void visit(ASTExpressionList & node, ASTPtr &, Data & data)
@ -267,10 +259,10 @@ struct AppendSemanticVisitorData
void visit(ASTSelectQuery & select, ASTPtr &)
{
if (done || !rev_aliases || !select.select_expression_list)
if (done || !rev_aliases || !select.select())
return;
for (auto & child : select.select_expression_list->children)
for (auto & child : select.select()->children)
{
if (auto * node = child->as<ASTAsterisk>())
AsteriskSemantic::setAliases(*node, rev_aliases);
@ -305,10 +297,10 @@ struct RewriteTablesVisitorData
bool needRewrite(ASTSelectQuery & select, std::vector<const ASTTableExpression *> & table_expressions)
{
if (!select.tables)
if (!select.tables())
return false;
const auto * tables = select.tables->as<ASTTablesInSelectQuery>();
const auto * tables = select.tables()->as<ASTTablesInSelectQuery>();
if (!tables)
return false;
@ -316,28 +308,47 @@ bool needRewrite(ASTSelectQuery & select, std::vector<const ASTTableExpression *
if (num_tables <= 2)
return false;
size_t num_array_join = 0;
size_t num_using = 0;
table_expressions.reserve(num_tables);
for (size_t i = 0; i < num_tables; ++i)
{
const auto * table = tables->children[i]->as<ASTTablesInSelectQueryElement>();
if (table && table->table_expression)
if (!table)
throw Exception("Table expected", ErrorCodes::LOGICAL_ERROR);
if (table->table_expression)
if (const auto * expression = table->table_expression->as<ASTTableExpression>())
table_expressions.push_back(expression);
if (!i)
continue;
if (!table || !table->table_join)
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
if (!table->table_join && !table->array_join)
throw Exception("Joined table expected", ErrorCodes::LOGICAL_ERROR);
if (table->array_join)
{
++num_array_join;
continue;
}
const auto & join = table->table_join->as<ASTTableJoin &>();
if (isComma(join.kind))
throw Exception("COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query", ErrorCodes::NOT_IMPLEMENTED);
/// it's not trivial to support mix of JOIN ON & JOIN USING cause of short names
if (join.using_expression_list)
throw Exception("Multiple JOIN does not support USING", ErrorCodes::NOT_IMPLEMENTED);
++num_using;
}
if (num_tables - num_array_join <= 2)
return false;
/// it's not trivial to support mix of JOIN ON & JOIN USING cause of short names
if (num_using)
throw Exception("Multiple JOIN does not support USING", ErrorCodes::NOT_IMPLEMENTED);
if (num_array_join)
throw Exception("Multiple JOIN does not support mix with ARRAY JOINs", ErrorCodes::NOT_IMPLEMENTED);
return true;
}
@ -369,21 +380,21 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
ExtractAsterisksVisitor(asterisks_data).visit(ast);
ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
if (select.select_expression_list)
if (select.select())
{
aliases_data.public_names = true;
ColumnAliasesVisitor(aliases_data).visit(select.select_expression_list);
ColumnAliasesVisitor(aliases_data).visit(select.refSelect());
aliases_data.public_names = false;
}
if (select.where_expression)
ColumnAliasesVisitor(aliases_data).visit(select.where_expression);
if (select.prewhere_expression)
ColumnAliasesVisitor(aliases_data).visit(select.prewhere_expression);
if (select.having_expression)
ColumnAliasesVisitor(aliases_data).visit(select.having_expression);
if (select.where())
ColumnAliasesVisitor(aliases_data).visit(select.refWhere());
if (select.prewhere())
ColumnAliasesVisitor(aliases_data).visit(select.refPrewhere());
if (select.having())
ColumnAliasesVisitor(aliases_data).visit(select.refHaving());
/// JOIN sections
for (auto & child : select.tables->children)
for (auto & child : select.tables()->children)
{
auto * table = child->as<ASTTablesInSelectQueryElement>();
if (table->table_join)
@ -399,7 +410,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
auto rev_aliases = std::make_shared<RevertedAliases>();
rev_aliases->swap(aliases_data.rev_aliases);
auto & src_tables = select.tables->children;
auto & src_tables = select.tables()->children;
ASTPtr left_table = src_tables[0];
for (size_t i = 1; i < src_tables.size() - 1; ++i)
@ -415,7 +426,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
/// replace tables in select with generated two-table join
RewriteVisitor::Data visitor_data{left_table, src_tables.back()};
RewriteVisitor(visitor_data).visit(select.tables);
RewriteVisitor(visitor_data).visit(select.refTables());
data.done = true;
}

View File

@ -43,7 +43,7 @@ void LogicalExpressionsOptimizer::perform()
return;
size_t position = 0;
for (auto & column : select_query->select_expression_list->children)
for (auto & column : select_query->select()->children)
{
bool inserted = column_to_position.emplace(column.get(), position).second;
@ -79,7 +79,7 @@ void LogicalExpressionsOptimizer::perform()
void LogicalExpressionsOptimizer::reorderColumns()
{
auto & columns = select_query->select_expression_list->children;
auto & columns = select_query->select()->children;
size_t cur_position = 0;
while (cur_position < columns.size())
@ -364,12 +364,12 @@ void LogicalExpressionsOptimizer::fixBrokenOrExpressions()
/// If the OR node was the root of the WHERE, PREWHERE, or HAVING expression, then update this root.
/// Due to the fact that we are dealing with a directed acyclic graph, we must check all cases.
if (select_query->where_expression && (or_function == &*(select_query->where_expression)))
select_query->where_expression = operands[0];
if (select_query->prewhere_expression && (or_function == &*(select_query->prewhere_expression)))
select_query->prewhere_expression = operands[0];
if (select_query->having_expression && (or_function == &*(select_query->having_expression)))
select_query->having_expression = operands[0];
if (select_query->where() && (or_function == &*(select_query->where())))
select_query->setExpression(ASTSelectQuery::Expression::WHERE, operands[0]->clone());
if (select_query->prewhere() && (or_function == &*(select_query->prewhere())))
select_query->setExpression(ASTSelectQuery::Expression::PREWHERE, operands[0]->clone());
if (select_query->having() && (or_function == &*(select_query->having())))
select_query->setExpression(ASTSelectQuery::Expression::HAVING, operands[0]->clone());
}
}
}

View File

@ -45,15 +45,14 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const
auto select = std::make_shared<ASTSelectQuery>();
select->select_expression_list = std::make_shared<ASTExpressionList>();
select->children.push_back(select->select_expression_list);
select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
auto count_func = std::make_shared<ASTFunction>();
count_func->name = "count";
count_func->arguments = std::make_shared<ASTExpressionList>();
select->select_expression_list->children.push_back(count_func);
select->select()->children.push_back(count_func);
if (commands.size() == 1)
select->where_expression = commands[0].predicate->clone();
select->setExpression(ASTSelectQuery::Expression::WHERE, commands[0].predicate->clone());
else
{
auto coalesced_predicates = std::make_shared<ASTFunction>();
@ -64,9 +63,8 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const
for (const MutationCommand & command : commands)
coalesced_predicates->arguments->children.push_back(command.predicate->clone());
select->where_expression = std::move(coalesced_predicates);
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(coalesced_predicates));
}
select->children.push_back(select->where_expression);
auto context_copy = context;
context_copy.getSettingsRef().merge_tree_uniform_read_distribution = 0;
@ -344,10 +342,9 @@ void MutationsInterpreter::prepare(bool dry_run)
auto select = std::make_shared<ASTSelectQuery>();
select->select_expression_list = std::make_shared<ASTExpressionList>();
select->children.push_back(select->select_expression_list);
select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
for (const auto & column_name : stages[0].output_columns)
select->select_expression_list->children.push_back(std::make_shared<ASTIdentifier>(column_name));
select->select()->children.push_back(std::make_shared<ASTIdentifier>(column_name));
if (!stages[0].filters.empty())
{
@ -363,8 +360,7 @@ void MutationsInterpreter::prepare(bool dry_run)
coalesced_predicates->arguments->children = stages[0].filters;
where_expression = std::move(coalesced_predicates);
}
select->where_expression = where_expression;
select->children.push_back(where_expression);
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression));
}
interpreter_select = std::make_unique<InterpreterSelectQuery>(select, context, storage, SelectQueryOptions().analyze(dry_run));

View File

@ -45,10 +45,10 @@ PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
bool PredicateExpressionsOptimizer::optimize()
{
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables || ast_select->tables->children.empty())
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables() || ast_select->tables()->children.empty())
return false;
if (!ast_select->where_expression && !ast_select->prewhere_expression)
if (!ast_select->where() && !ast_select->prewhere())
return false;
SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns();
@ -56,15 +56,15 @@ bool PredicateExpressionsOptimizer::optimize()
bool is_rewrite_subqueries = false;
if (!all_subquery_projection_columns.empty())
{
is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE);
is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE);
is_rewrite_subqueries |= optimizeImpl(ast_select->where(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE);
is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE);
}
return is_rewrite_subqueries;
}
bool PredicateExpressionsOptimizer::optimizeImpl(
ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind)
{
/// split predicate with `and`
std::vector<ASTPtr> outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
@ -99,9 +99,15 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
switch (optimize_kind)
{
case OptimizeKind::NONE: continue;
case OptimizeKind::PUSH_TO_WHERE: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->where_expression, subquery); continue;
case OptimizeKind::PUSH_TO_HAVING: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->having_expression, subquery); continue;
case OptimizeKind::PUSH_TO_PREWHERE: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->prewhere_expression, subquery); continue;
case OptimizeKind::PUSH_TO_WHERE:
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::WHERE);
continue;
case OptimizeKind::PUSH_TO_HAVING:
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::HAVING);
continue;
case OptimizeKind::PUSH_TO_PREWHERE:
is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::PREWHERE);
continue;
}
}
}
@ -111,9 +117,9 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
bool PredicateExpressionsOptimizer::allowPushDown(const ASTSelectQuery * subquery)
{
if (subquery && !subquery->final() && !subquery->limit_by_expression_list && !subquery->limit_length && !subquery->with_expression_list)
if (subquery && !subquery->final() && !subquery->limitBy() && !subquery->limitLength() && !subquery->with())
{
ASTPtr expr_list = ast_select->select_expression_list;
ASTPtr expr_list = ast_select->select();
ExtractFunctionVisitor::Data extract_data;
ExtractFunctionVisitor(extract_data).visit(expr_list);
@ -132,7 +138,7 @@ bool PredicateExpressionsOptimizer::allowPushDown(const ASTSelectQuery * subquer
return false;
}
std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression)
std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(const ASTPtr & predicate_expression)
{
std::vector<ASTPtr> predicate_expressions;
@ -273,19 +279,13 @@ bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
return false;
}
bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery)
bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery,
ASTSelectQuery::Expression expr)
{
ASTPtr new_subquery_expression = subquery_expression;
new_subquery_expression = new_subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression;
ASTPtr subquery_expression = subquery->getExpression(expr, false);
subquery_expression = subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression;
if (!subquery_expression)
subquery->children.emplace_back(new_subquery_expression);
else
for (auto & child : subquery->children)
if (child == subquery_expression)
child = new_subquery_expression;
subquery_expression = std::move(new_subquery_expression);
subquery->setExpression(expr, std::move(subquery_expression));
return true;
}
@ -351,7 +351,7 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
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()->children)
{
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>())
{
@ -371,7 +371,7 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk)
{
/// SELECT *, SELECT dummy, SELECT 1 AS id
if (!select_query->tables || select_query->tables->children.empty())
if (!select_query->tables() || select_query->tables()->children.empty())
return {};
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);

View File

@ -2,13 +2,13 @@
#include "DatabaseAndTableWithAlias.h"
#include "ExpressionAnalyzer.h"
#include <Parsers/ASTSelectQuery.h>
#include <map>
namespace DB
{
class ASTIdentifier;
class ASTSelectQuery;
class ASTSubquery;
class Context;
@ -69,14 +69,14 @@ private:
bool isArrayJoinFunction(const ASTPtr & node);
std::vector<ASTPtr> splitConjunctionPredicate(ASTPtr & predicate_expression);
std::vector<ASTPtr> splitConjunctionPredicate(const ASTPtr & predicate_expression);
std::vector<IdentifierWithQualifier> getDependenciesAndQualifiers(ASTPtr & expression,
std::vector<TableWithColumnNames> & tables_with_aliases);
bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery);
bool optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery, ASTSelectQuery::Expression expr);
bool optimizeImpl(ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind);
bool optimizeImpl(const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind);
bool allowPushDown(const ASTSelectQuery * subquery);

View File

@ -155,20 +155,22 @@ static bool needVisitChild(const ASTPtr & child)
}
/// special visitChildren() for ASTSelectQuery
void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr & ast, Data & data)
void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data)
{
for (auto & child : ast->children)
for (auto & child : select.children)
if (needVisitChild(child))
visit(child, data);
#if 1 /// TODO: legacy?
/// 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);
if (select.prewhere())
visit(select.refPrewhere(), data);
if (select.where())
visit(select.refWhere(), data);
if (select.having())
visit(select.refHaving(), data);
#endif
}
/// Don't go into subqueries.

View File

@ -114,7 +114,7 @@ void RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & data)
void RequiredSourceColumnsMatcher::visit(ASTSelectQuery & select, const ASTPtr &, Data & data)
{
/// special case for top-level SELECT items: they are publics
for (auto & node : select.select_expression_list->children)
for (auto & node : select.select()->children)
{
if (const auto * identifier = node->as<ASTIdentifier>())
data.addColumnIdentifier(*identifier);
@ -124,14 +124,11 @@ void RequiredSourceColumnsMatcher::visit(ASTSelectQuery & select, const ASTPtr &
std::vector<ASTPtr *> out;
for (auto & node : select.children)
if (node != select.select_expression_list)
out.push_back(&node);
if (node != select.select())
Visitor(data).visit(node);
/// revisit select_expression_list (with children) when all the aliases are set
out.push_back(&select.select_expression_list);
for (ASTPtr * add_node : out)
Visitor(data).visit(*add_node);
Visitor(data).visit(select.refSelect());
}
void RequiredSourceColumnsMatcher::visit(const ASTIdentifier & node, const ASTPtr &, Data & data)

View File

@ -126,7 +126,7 @@ bool hasArrayJoin(const ASTPtr & ast)
/// Keep number of columns for 'GLOBAL IN (SELECT 1 AS a, a)'
void renameDuplicatedColumns(const ASTSelectQuery * select_query)
{
ASTs & elements = select_query->select_expression_list->children;
ASTs & elements = select_query->select()->children;
std::set<String> all_column_names;
std::set<String> assigned_column_names;
@ -159,7 +159,7 @@ void renameDuplicatedColumns(const ASTSelectQuery * select_query)
/// Also we have to remove duplicates in case of GLOBAL subqueries. Their results are placed into tables so duplicates are inpossible.
void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns, bool remove_dups)
{
ASTs & elements = select_query->select_expression_list->children;
ASTs & elements = select_query->select()->children;
std::map<String, size_t> required_columns_with_duplicate_count;
@ -255,7 +255,7 @@ const std::unordered_set<String> possibly_injective_function_names
/// Eliminates injective function calls and constant expressions from group by statement.
void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context)
{
if (!select_query->group_expression_list)
if (!select_query->groupBy())
return;
const auto is_literal = [] (const ASTPtr & ast) -> bool
@ -263,7 +263,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
return ast->as<ASTLiteral>();
};
auto & group_exprs = select_query->group_expression_list->children;
auto & group_exprs = select_query->groupBy()->children;
/// removes expression at index idx by making it last one and calling .pop_back()
const auto remove_expr_at_index = [&group_exprs] (const size_t idx)
@ -347,22 +347,22 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
unused_column_name = toString(unused_column);
}
select_query->group_expression_list = std::make_shared<ASTExpressionList>();
select_query->group_expression_list->children.emplace_back(std::make_shared<ASTLiteral>(UInt64(unused_column)));
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::make_shared<ASTExpressionList>());
select_query->groupBy()->children.emplace_back(std::make_shared<ASTLiteral>(UInt64(unused_column)));
}
}
/// Remove duplicate items from ORDER BY.
void optimizeOrderBy(const ASTSelectQuery * select_query)
{
if (!select_query->order_expression_list)
if (!select_query->orderBy())
return;
/// Make unique sorting conditions.
using NameAndLocale = std::pair<String, String>;
std::set<NameAndLocale> elems_set;
ASTs & elems = select_query->order_expression_list->children;
ASTs & elems = select_query->orderBy()->children;
ASTs unique_elems;
unique_elems.reserve(elems.size());
@ -376,18 +376,18 @@ void optimizeOrderBy(const ASTSelectQuery * select_query)
}
if (unique_elems.size() < elems.size())
elems = unique_elems;
elems = std::move(unique_elems);
}
/// Remove duplicate items from LIMIT BY.
void optimizeLimitBy(const ASTSelectQuery * select_query)
{
if (!select_query->limit_by_expression_list)
if (!select_query->limitBy())
return;
std::set<String> elems_set;
ASTs & elems = select_query->limit_by_expression_list->children;
ASTs & elems = select_query->limitBy()->children;
ASTs unique_elems;
unique_elems.reserve(elems.size());
@ -398,7 +398,7 @@ void optimizeLimitBy(const ASTSelectQuery * select_query)
}
if (unique_elems.size() < elems.size())
elems = unique_elems;
elems = std::move(unique_elems);
}
/// Remove duplicated columns from USING(...).

View File

@ -113,14 +113,10 @@ void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk & , const
void TranslateQualifiedNamesMatcher::visit(ASTTableJoin & join, const ASTPtr & , Data & data)
{
std::vector<ASTPtr *> out;
if (join.using_expression_list)
out.push_back(&join.using_expression_list);
Visitor(data).visit(join.using_expression_list);
else if (join.on_expression)
out.push_back(&join.on_expression);
for (ASTPtr * add_node : out)
Visitor(data).visit(*add_node);
Visitor(data).visit(join.on_expression);
}
void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr & , Data & data)
@ -128,18 +124,16 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
if (auto join = select.join())
extractJoinUsingColumns(join->table_join, data);
#if 1 /// TODO: legacy?
/// If the WHERE clause or HAVING consists of a single qualified column, the reference must be translated not only in children,
/// but also in where_expression and having_expression.
std::vector<ASTPtr *> out;
if (select.prewhere_expression)
out.push_back(&select.prewhere_expression);
if (select.where_expression)
out.push_back(&select.where_expression);
if (select.having_expression)
out.push_back(&select.having_expression);
for (ASTPtr * add_node : out)
Visitor(data).visit(*add_node);
if (select.prewhere())
Visitor(data).visit(select.refPrewhere());
if (select.where())
Visitor(data).visit(select.refWhere());
if (select.having())
Visitor(data).visit(select.refHaving());
#endif
}
static void addIdentifier(ASTs & nodes, const String & table_name, const String & column_name, AsteriskSemantic::RevertedAliasesPtr aliases)

View File

@ -55,9 +55,8 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const auto select_query = std::make_shared<ASTSelectQuery>();
select_with_union_query->list_of_selects->children.push_back(select_query);
const auto select_expression_list = std::make_shared<ASTExpressionList>();
select_query->select_expression_list = select_expression_list;
select_query->children.emplace_back(select_query->select_expression_list);
select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
const auto select_expression_list = select_query->select();
NamesAndTypesList columns;

View File

@ -281,9 +281,9 @@ void reorder(DB::IAST * ast)
if (select_query == nullptr)
return;
reorderImpl(select_query->where_expression.get());
reorderImpl(select_query->prewhere_expression.get());
reorderImpl(select_query->having_expression.get());
reorderImpl(select_query->where().get());
reorderImpl(select_query->prewhere().get());
reorderImpl(select_query->having().get());
}
}

View File

@ -13,7 +13,6 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
@ -21,8 +20,9 @@ ASTPtr ASTSelectQuery::clone() const
{
auto res = std::make_shared<ASTSelectQuery>(*this);
res->children.clear();
res->positions.clear();
#define CLONE(member) if (member) { res->member = member->clone(); res->children.push_back(res->member); }
#define CLONE(expr) res->setExpression(expr, getExpression(expr, true))
/** NOTE Members must clone exactly in the same order,
* in which they were inserted into `children` in ParserSelectQuery.
@ -34,19 +34,19 @@ ASTPtr ASTSelectQuery::clone() const
* And if the cloning order does not match the parsing order,
* then different servers will get different identifiers.
*/
CLONE(with_expression_list)
CLONE(select_expression_list)
CLONE(tables)
CLONE(prewhere_expression)
CLONE(where_expression)
CLONE(group_expression_list)
CLONE(having_expression)
CLONE(order_expression_list)
CLONE(limit_by_value)
CLONE(limit_by_expression_list)
CLONE(limit_offset)
CLONE(limit_length)
CLONE(settings)
CLONE(Expression::WITH);
CLONE(Expression::SELECT);
CLONE(Expression::TABLES);
CLONE(Expression::PREWHERE);
CLONE(Expression::WHERE);
CLONE(Expression::GROUP_BY);
CLONE(Expression::HAVING);
CLONE(Expression::ORDER_BY);
CLONE(Expression::LIMIT_BY_VALUE);
CLONE(Expression::LIMIT_BY);
CLONE(Expression::LIMIT_OFFSET);
CLONE(Expression::LIMIT_LENGTH);
CLONE(Expression::SETTINGS);
#undef CLONE
@ -60,45 +60,45 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
frame.need_parens = false;
std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' ');
if (with_expression_list)
if (with())
{
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : "");
s.one_line
? with_expression_list->formatImpl(s, state, frame)
: with_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
? with()->formatImpl(s, state, frame)
: with()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
s.ostr << s.nl_or_ws;
}
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (distinct ? "DISTINCT " : "") << (s.hilite ? hilite_none : "");
s.one_line
? select_expression_list->formatImpl(s, state, frame)
: select_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
? select()->formatImpl(s, state, frame)
: select()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
if (tables)
if (tables())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FROM " << (s.hilite ? hilite_none : "");
tables->formatImpl(s, state, frame);
tables()->formatImpl(s, state, frame);
}
if (prewhere_expression)
if (prewhere())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "PREWHERE " << (s.hilite ? hilite_none : "");
prewhere_expression->formatImpl(s, state, frame);
prewhere()->formatImpl(s, state, frame);
}
if (where_expression)
if (where())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "WHERE " << (s.hilite ? hilite_none : "");
where_expression->formatImpl(s, state, frame);
where()->formatImpl(s, state, frame);
}
if (group_expression_list)
if (groupBy())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
s.one_line
? group_expression_list->formatImpl(s, state, frame)
: group_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
? groupBy()->formatImpl(s, state, frame)
: groupBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
}
if (group_by_with_rollup)
@ -110,45 +110,45 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
if (group_by_with_totals)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : "");
if (having_expression)
if (having())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "HAVING " << (s.hilite ? hilite_none : "");
having_expression->formatImpl(s, state, frame);
having()->formatImpl(s, state, frame);
}
if (order_expression_list)
if (orderBy())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
s.one_line
? order_expression_list->formatImpl(s, state, frame)
: order_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
? orderBy()->formatImpl(s, state, frame)
: orderBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
}
if (limit_by_value)
if (limitByValue())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "LIMIT " << (s.hilite ? hilite_none : "");
limit_by_value->formatImpl(s, state, frame);
limitByValue()->formatImpl(s, state, frame);
s.ostr << (s.hilite ? hilite_keyword : "") << " BY " << (s.hilite ? hilite_none : "");
s.one_line
? limit_by_expression_list->formatImpl(s, state, frame)
: limit_by_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
? limitBy()->formatImpl(s, state, frame)
: limitBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
}
if (limit_length)
if (limitLength())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "LIMIT " << (s.hilite ? hilite_none : "");
if (limit_offset)
if (limitOffset())
{
limit_offset->formatImpl(s, state, frame);
limitOffset()->formatImpl(s, state, frame);
s.ostr << ", ";
}
limit_length->formatImpl(s, state, frame);
limitLength()->formatImpl(s, state, frame);
}
if (settings)
if (settings())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : "");
settings->formatImpl(s, state, frame);
settings()->formatImpl(s, state, frame);
}
}
@ -158,10 +158,10 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
static const ASTTableExpression * getFirstTableExpression(const ASTSelectQuery & select)
{
if (!select.tables)
if (!select.tables())
return {};
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
const auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
if (tables_in_select_query.children.empty())
return {};
@ -174,10 +174,10 @@ static const ASTTableExpression * getFirstTableExpression(const ASTSelectQuery &
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select)
{
if (!select.tables)
if (!select.tables())
return {};
auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
if (tables_in_select_query.children.empty())
return {};
@ -190,10 +190,10 @@ static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select)
static const ASTArrayJoin * getFirstArrayJoin(const ASTSelectQuery & select)
{
if (!select.tables)
if (!select.tables())
return {};
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
const auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
if (tables_in_select_query.children.empty())
return {};
@ -215,10 +215,10 @@ static const ASTArrayJoin * getFirstArrayJoin(const ASTSelectQuery & select)
static const ASTTablesInSelectQueryElement * getFirstTableJoin(const ASTSelectQuery & select)
{
if (!select.tables)
if (!select.tables())
return {};
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
const auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
if (tables_in_select_query.children.empty())
return {};
@ -311,14 +311,12 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
if (!table_expression)
{
auto tables_list = std::make_shared<ASTTablesInSelectQuery>();
setExpression(Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
auto element = std::make_shared<ASTTablesInSelectQueryElement>();
auto table_expr = std::make_shared<ASTTableExpression>();
element->table_expression = table_expr;
element->children.emplace_back(table_expr);
tables_list->children.emplace_back(element);
tables = tables_list;
children.emplace_back(tables_list);
tables()->children.emplace_back(element);
table_expression = table_expr.get();
}
@ -336,14 +334,12 @@ void ASTSelectQuery::addTableFunction(ASTPtr & table_function_ptr)
if (!table_expression)
{
auto tables_list = std::make_shared<ASTTablesInSelectQuery>();
setExpression(Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
auto element = std::make_shared<ASTTablesInSelectQueryElement>();
auto table_expr = std::make_shared<ASTTableExpression>();
element->table_expression = table_expr;
element->children.emplace_back(table_expr);
tables_list->children.emplace_back(element);
tables = tables_list;
children.emplace_back(tables_list);
tables()->children.emplace_back(element);
table_expression = table_expr.get();
}
@ -356,4 +352,35 @@ void ASTSelectQuery::addTableFunction(ASTPtr & table_function_ptr)
table_expression->table_function->setAlias(table_alias);
}
void ASTSelectQuery::setExpression(Expression expr, ASTPtr && ast)
{
if (ast)
{
auto it = positions.find(expr);
if (it == positions.end())
{
positions[expr] = children.size();
children.emplace_back(ast);
}
else
children[it->second] = ast;
}
else if (positions.count(expr))
{
size_t pos = positions[expr];
children.erase(children.begin() + pos);
positions.erase(expr);
for (auto & pr : positions)
if (pr.second > pos)
--pr.second;
}
}
ASTPtr & ASTSelectQuery::getExpression(Expression expr)
{
if (!positions.count(expr))
throw Exception("Get expression before set", ErrorCodes::LOGICAL_ERROR);
return children[positions[expr]];
}
}

View File

@ -15,28 +15,63 @@ struct ASTTablesInSelectQueryElement;
class ASTSelectQuery : public IAST
{
public:
enum class Expression : UInt8
{
WITH,
SELECT,
TABLES,
PREWHERE,
WHERE,
GROUP_BY,
HAVING,
ORDER_BY,
LIMIT_BY_VALUE,
LIMIT_BY,
LIMIT_OFFSET,
LIMIT_LENGTH,
SETTINGS
};
/** Get the text that identifies this element. */
String getID(char) const override { return "SelectQuery"; }
ASTPtr clone() const override;
bool distinct = false;
ASTPtr with_expression_list;
ASTPtr select_expression_list;
ASTPtr tables; // pointer to TablesInSelectQuery
ASTPtr prewhere_expression;
ASTPtr where_expression;
ASTPtr group_expression_list;
bool group_by_with_totals = false;
bool group_by_with_rollup = false;
bool group_by_with_cube = false;
ASTPtr having_expression;
ASTPtr order_expression_list;
ASTPtr limit_by_value;
ASTPtr limit_by_expression_list;
ASTPtr limit_offset;
ASTPtr limit_length;
ASTPtr settings;
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }
ASTPtr & refTables() { return getExpression(Expression::TABLES); }
ASTPtr & refPrewhere() { return getExpression(Expression::PREWHERE); }
ASTPtr & refWhere() { return getExpression(Expression::WHERE); }
ASTPtr & refHaving() { return getExpression(Expression::HAVING); }
const ASTPtr with() const { return getExpression(Expression::WITH); }
const ASTPtr select() const { return getExpression(Expression::SELECT); }
const ASTPtr tables() const { return getExpression(Expression::TABLES); }
const ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); }
const ASTPtr where() const { return getExpression(Expression::WHERE); }
const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
const ASTPtr having() const { return getExpression(Expression::HAVING); }
const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
const ASTPtr limitByValue() const { return getExpression(Expression::LIMIT_BY_VALUE); }
const ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); }
const ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); }
const ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); }
const ASTPtr settings() const { return getExpression(Expression::SETTINGS); }
/// Set/Reset/Remove expression.
void setExpression(Expression expr, ASTPtr && ast);
ASTPtr getExpression(Expression expr, bool clone = false) const
{
auto it = positions.find(expr);
if (it != positions.end())
return clone ? children[it->second]->clone() : children[it->second];
return {};
}
/// Compatibility with old parser of tables list. TODO remove
ASTPtr sample_size() const;
@ -50,6 +85,11 @@ public:
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
private:
std::unordered_map<Expression, size_t> positions;
ASTPtr & getExpression(Expression expr);
};
}

View File

@ -52,11 +52,25 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
ASTPtr with_expression_list;
ASTPtr select_expression_list;
ASTPtr tables;
ASTPtr prewhere_expression;
ASTPtr where_expression;
ASTPtr group_expression_list;
ASTPtr having_expression;
ASTPtr order_expression_list;
ASTPtr limit_by_value;
ASTPtr limit_by_expression_list;
ASTPtr limit_offset;
ASTPtr limit_length;
ASTPtr settings;
/// WITH expr list
{
if (s_with.ignore(pos, expected))
{
if (!exp_list_for_with_clause.parse(pos, select_query->with_expression_list, expected))
if (!exp_list_for_with_clause.parse(pos, with_expression_list, expected))
return false;
}
}
@ -75,40 +89,40 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (open_bracket.ignore(pos, expected))
{
if (!num.parse(pos, select_query->limit_length, expected))
if (!num.parse(pos, limit_length, expected))
return false;
if (!close_bracket.ignore(pos, expected))
return false;
}
else
{
if (!num.parse(pos, select_query->limit_length, expected))
if (!num.parse(pos, limit_length, expected))
return false;
}
}
if (!exp_list_for_select_clause.parse(pos, select_query->select_expression_list, expected))
if (!exp_list_for_select_clause.parse(pos, select_expression_list, expected))
return false;
}
/// FROM database.table or FROM table or FROM (subquery) or FROM tableFunction(...)
if (s_from.ignore(pos, expected))
{
if (!ParserTablesInSelectQuery().parse(pos, select_query->tables, expected))
if (!ParserTablesInSelectQuery().parse(pos, tables, expected))
return false;
}
/// PREWHERE expr
if (s_prewhere.ignore(pos, expected))
{
if (!exp_elem.parse(pos, select_query->prewhere_expression, expected))
if (!exp_elem.parse(pos, prewhere_expression, expected))
return false;
}
/// WHERE expr
if (s_where.ignore(pos, expected))
{
if (!exp_elem.parse(pos, select_query->where_expression, expected))
if (!exp_elem.parse(pos, where_expression, expected))
return false;
}
@ -123,7 +137,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !open_bracket.ignore(pos, expected))
return false;
if (!exp_list.parse(pos, select_query->group_expression_list, expected))
if (!exp_list.parse(pos, group_expression_list, expected))
return false;
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !close_bracket.ignore(pos, expected))
@ -155,45 +169,45 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
/// HAVING expr
if (s_having.ignore(pos, expected))
{
if (!exp_elem.parse(pos, select_query->having_expression, expected))
if (!exp_elem.parse(pos, having_expression, expected))
return false;
}
/// ORDER BY expr ASC|DESC COLLATE 'locale' list
if (s_order_by.ignore(pos, expected))
{
if (!order_list.parse(pos, select_query->order_expression_list, expected))
if (!order_list.parse(pos, order_expression_list, expected))
return false;
}
/// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list
if (s_limit.ignore(pos, expected))
{
if (select_query->limit_length)
if (limit_length)
throw Exception("Can not use TOP and LIMIT together", ErrorCodes::TOP_AND_LIMIT_TOGETHER);
ParserToken s_comma(TokenType::Comma);
if (!exp_elem.parse(pos, select_query->limit_length, expected))
if (!exp_elem.parse(pos, limit_length, expected))
return false;
if (s_comma.ignore(pos, expected))
{
select_query->limit_offset = select_query->limit_length;
if (!exp_elem.parse(pos, select_query->limit_length, expected))
limit_offset = limit_length;
if (!exp_elem.parse(pos, limit_length, expected))
return false;
}
else if (s_by.ignore(pos, expected))
{
select_query->limit_by_value = select_query->limit_length;
select_query->limit_length = nullptr;
limit_by_value = limit_length;
limit_length = nullptr;
if (!exp_list.parse(pos, select_query->limit_by_expression_list, expected))
if (!exp_list.parse(pos, limit_by_expression_list, expected))
return false;
}
else if (s_offset.ignore(pos, expected))
{
if (!exp_elem.parse(pos, select_query->limit_offset, expected))
if (!exp_elem.parse(pos, limit_offset, expected))
return false;
}
}
@ -201,18 +215,18 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
/// LIMIT length | LIMIT offset, length
if (s_limit.ignore(pos, expected))
{
if (!select_query->limit_by_value || select_query->limit_length)
if (!limit_by_value || limit_length)
return false;
ParserToken s_comma(TokenType::Comma);
if (!exp_elem.parse(pos, select_query->limit_length, expected))
if (!exp_elem.parse(pos, limit_length, expected))
return false;
if (s_comma.ignore(pos, expected))
{
select_query->limit_offset = select_query->limit_length;
if (!exp_elem.parse(pos, select_query->limit_length, expected))
limit_offset = limit_length;
if (!exp_elem.parse(pos, limit_length, expected))
return false;
}
}
@ -222,36 +236,23 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserSetQuery parser_settings(true);
if (!parser_settings.parse(pos, select_query->settings, expected))
if (!parser_settings.parse(pos, settings, expected))
return false;
}
if (select_query->with_expression_list)
select_query->children.push_back(select_query->with_expression_list);
select_query->children.push_back(select_query->select_expression_list);
if (select_query->tables)
select_query->children.push_back(select_query->tables);
if (select_query->prewhere_expression)
select_query->children.push_back(select_query->prewhere_expression);
if (select_query->where_expression)
select_query->children.push_back(select_query->where_expression);
if (select_query->group_expression_list)
select_query->children.push_back(select_query->group_expression_list);
if (select_query->having_expression)
select_query->children.push_back(select_query->having_expression);
if (select_query->order_expression_list)
select_query->children.push_back(select_query->order_expression_list);
if (select_query->limit_by_value)
select_query->children.push_back(select_query->limit_by_value);
if (select_query->limit_by_expression_list)
select_query->children.push_back(select_query->limit_by_expression_list);
if (select_query->limit_offset)
select_query->children.push_back(select_query->limit_offset);
if (select_query->limit_length)
select_query->children.push_back(select_query->limit_length);
if (select_query->settings)
select_query->children.push_back(select_query->settings);
select_query->setExpression(ASTSelectQuery::Expression::WITH, std::move(with_expression_list));
select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables));
select_query->setExpression(ASTSelectQuery::Expression::PREWHERE, std::move(prewhere_expression));
select_query->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression));
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::move(group_expression_list));
select_query->setExpression(ASTSelectQuery::Expression::HAVING, std::move(having_expression));
select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, std::move(order_expression_list));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_VALUE, std::move(limit_by_value));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY, std::move(limit_by_expression_list));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(limit_offset));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(limit_length));
select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings));
return true;
}

View File

@ -285,19 +285,19 @@ KeyCondition::KeyCondition(
/// Trasform WHERE section to Reverse Polish notation
const auto & select = query_info.query->as<ASTSelectQuery &>();
if (select.where_expression)
if (select.where())
{
traverseAST(select.where_expression, context, block_with_constants);
traverseAST(select.where(), context, block_with_constants);
if (select.prewhere_expression)
if (select.prewhere())
{
traverseAST(select.prewhere_expression, context, block_with_constants);
traverseAST(select.prewhere(), context, block_with_constants);
rpn.emplace_back(RPNElement::FUNCTION_AND);
}
}
else if (select.prewhere_expression)
else if (select.prewhere())
{
traverseAST(select.prewhere_expression, context, block_with_constants);
traverseAST(select.prewhere(), context, block_with_constants);
}
else
{

View File

@ -512,8 +512,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
/// PREWHERE
String prewhere_column;
if (select.prewhere_expression)
prewhere_column = select.prewhere_expression->getColumnName();
if (select.prewhere())
prewhere_column = select.prewhere()->getColumnName();
RangesInDataParts parts_with_ranges;

View File

@ -229,15 +229,15 @@ SetIndexCondition::SetIndexCondition(
/// Replace logical functions with bit functions.
/// Working with UInt8: last bit = can be true, previous = can be false.
if (select.where_expression && select.prewhere_expression)
if (select.where() && select.prewhere())
expression_ast = makeASTFunction(
"and",
select.where_expression->clone(),
select.prewhere_expression->clone());
else if (select.where_expression)
expression_ast = select.where_expression->clone();
else if (select.prewhere_expression)
expression_ast = select.prewhere_expression->clone();
select.where()->clone(),
select.prewhere()->clone());
else if (select.where())
expression_ast = select.where()->clone();
else if (select.prewhere())
expression_ast = select.prewhere()->clone();
else
expression_ast = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);

View File

@ -139,10 +139,10 @@ ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions) const
void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
{
if (!select.where_expression || select.prewhere_expression)
if (!select.where() || select.prewhere())
return;
Conditions where_conditions = analyze(select.where_expression);
Conditions where_conditions = analyze(select.where());
Conditions prewhere_conditions;
UInt64 total_size_of_moved_conditions = 0;
@ -186,21 +186,10 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
/// Rewrite the SELECT query.
auto old_where = std::find(std::begin(select.children), std::end(select.children), select.where_expression);
if (old_where == select.children.end())
throw Exception("Logical error: cannot find WHERE expression in the list of children of SELECT query", ErrorCodes::LOGICAL_ERROR);
select.setExpression(ASTSelectQuery::Expression::WHERE, reconstruct(where_conditions));
select.setExpression(ASTSelectQuery::Expression::PREWHERE, reconstruct(prewhere_conditions));
select.where_expression = reconstruct(where_conditions);
select.prewhere_expression = reconstruct(prewhere_conditions);
if (select.where_expression)
*old_where = select.where_expression;
else
select.children.erase(old_where);
select.children.push_back(select.prewhere_expression);
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"" << select.prewhere_expression << "\" moved to PREWHERE");
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"" << select.prewhere() << "\" moved to PREWHERE");
}

View File

@ -37,19 +37,19 @@ public:
/// Trasform WHERE section to Reverse Polish notation
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query_info.query);
if (select.where_expression)
if (select.where())
{
traverseAST(select.where_expression);
traverseAST(select.where());
if (select.prewhere_expression)
if (select.prewhere())
{
traverseAST(select.prewhere_expression);
traverseAST(select.prewhere());
rpn.emplace_back(RPNElement::FUNCTION_AND);
}
}
else if (select.prewhere_expression)
else if (select.prewhere())
{
traverseAST(select.prewhere_expression);
traverseAST(select.prewhere());
}
else
{

View File

@ -65,8 +65,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
PartitionCommand res;
res.type = CLEAR_COLUMN;
res.partition = command_ast->partition;
const Field & column_name = *getIdentifierName(command_ast->column);
res.column_name = column_name;
res.column_name = *getIdentifierName(command_ast->column);
return res;
}
else if (command_ast->type == ASTAlterCommand::FREEZE_ALL)

View File

@ -471,12 +471,10 @@ ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const Select
{
const auto & select = query_info.query->as<ASTSelectQuery &>();
if (!select.where_expression)
{
if (!select.where())
return nullptr;
}
const auto & blocks = evaluateExpressionOverConstantCondition(select.where_expression, sharding_key_expr);
const auto & blocks = evaluateExpressionOverConstantCondition(select.where(), sharding_key_expr);
// Can't get definite answer if we can skip any shards
if (!blocks)

View File

@ -66,7 +66,7 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa
static void checkAllowedQueries(const ASTSelectQuery & query)
{
if (query.prewhere_expression || query.final() || query.sample_size())
if (query.prewhere() || query.final() || query.sample_size())
throw Exception("MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL.", DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
ASTPtr subquery = extractTableExpression(query, 0);

View File

@ -369,7 +369,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
{
StoragePtr storage = iterator->table();
if (query && query->as<ASTSelectQuery>()->prewhere_expression && !storage->supportsPrewhere())
if (query && query->as<ASTSelectQuery>()->prewhere() && !storage->supportsPrewhere())
throw Exception("Storage " + storage->getName() + " doesn't support PREWHERE.", ErrorCodes::ILLEGAL_PREWHERE);
if (storage.get() != this)
@ -440,7 +440,7 @@ void StorageMerge::convertingSourceStream(const Block & header, const Context &
Block before_block_header = source_stream->getHeader();
source_stream = std::make_shared<ConvertingBlockInputStream>(context, source_stream, header, ConvertingBlockInputStream::MatchColumnsMode::Name);
auto where_expression = query->as<ASTSelectQuery>()->where_expression;
auto where_expression = query->as<ASTSelectQuery>()->where();
if (!where_expression)
return;

View File

@ -75,7 +75,7 @@ BlockInputStreams StorageView::read(
void StorageView::replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery)
{
auto * select_element = select_query->tables->children[0]->as<ASTTablesInSelectQueryElement>();
auto * select_element = select_query->tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);

View File

@ -96,11 +96,11 @@ static bool extractPathImpl(const IAST & elem, String & res)
static String extractPath(const ASTPtr & query)
{
const auto & select = query->as<ASTSelectQuery &>();
if (!select.where_expression)
if (!select.where())
return "";
String res;
return extractPathImpl(*select.where_expression, res) ? res : "";
return extractPathImpl(*select.where(), res) ? res : "";
}

View File

@ -77,16 +77,13 @@ String chooseSuffixForSet(const NamesAndTypesList & columns, const std::vector<S
void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value)
{
auto & select = ast->as<ASTSelectQuery &>();
if (!select.with_expression_list)
{
select.with_expression_list = std::make_shared<ASTExpressionList>();
select.children.insert(select.children.begin(), select.with_expression_list);
}
if (!select.with())
select.setExpression(ASTSelectQuery::Expression::WITH, std::make_shared<ASTExpressionList>());
auto literal = std::make_shared<ASTLiteral>(value);
literal->alias = column_name;
literal->prefer_alias_to_column_name = true;
select.with_expression_list->children.push_back(literal);
select.with()->children.push_back(literal);
}
/// Verifying that the function depends only on the specified columns
@ -136,7 +133,7 @@ static ASTPtr buildWhereExpression(const ASTs & functions)
void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context)
{
const auto & select = query->as<ASTSelectQuery &>();
if (!select.where_expression && !select.prewhere_expression)
if (!select.where() && !select.prewhere())
return;
NameSet columns;
@ -145,10 +142,10 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c
/// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.
std::vector<ASTPtr> functions;
if (select.where_expression)
extractFunctions(select.where_expression, columns, functions);
if (select.prewhere_expression)
extractFunctions(select.prewhere_expression, columns, functions);
if (select.where())
extractFunctions(select.where(), columns, functions);
if (select.prewhere())
extractFunctions(select.prewhere(), columns, functions);
ASTPtr expression_ast = buildWhereExpression(functions);
if (!expression_ast)

View File

@ -104,7 +104,7 @@ String transformQueryForExternalDatabase(
for (const auto & name : used_columns)
select_expr_list->children.push_back(std::make_shared<ASTIdentifier>(name));
select->select_expression_list = std::move(select_expr_list);
select->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expr_list));
/** If there was WHERE,
* copy it to transformed query if it is compatible,
@ -112,13 +112,13 @@ String transformQueryForExternalDatabase(
* copy only compatible parts of it.
*/
auto & original_where = clone_query->as<ASTSelectQuery &>().where_expression;
ASTPtr original_where = clone_query->as<ASTSelectQuery &>().where();
if (original_where)
{
replaceConstFunction(*original_where, context, available_columns);
if (isCompatible(*original_where))
{
select->where_expression = original_where;
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where));
}
else if (const auto * function = original_where->as<ASTFunction>())
{
@ -140,7 +140,7 @@ String transformQueryForExternalDatabase(
}
if (compatible_found)
select->where_expression = std::move(new_function_and);
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(new_function_and));
}
}
}

View File

@ -17,19 +17,11 @@ from subprocess import CalledProcessError
from datetime import datetime
from time import sleep
from errno import ESRCH
from termcolor import colored
import termcolor
from random import random
import commands
OP_SQUARE_BRACKET = colored("[", attrs=['bold'])
CL_SQUARE_BRACKET = colored("]", attrs=['bold'])
MSG_FAIL = OP_SQUARE_BRACKET + colored(" FAIL ", "red", attrs=['bold']) + CL_SQUARE_BRACKET
MSG_UNKNOWN = OP_SQUARE_BRACKET + colored(" UNKNOWN ", "yellow", attrs=['bold']) + CL_SQUARE_BRACKET
MSG_OK = OP_SQUARE_BRACKET + colored(" OK ", "green", attrs=['bold']) + CL_SQUARE_BRACKET
MSG_SKIPPED = OP_SQUARE_BRACKET + colored(" SKIPPED ", "cyan", attrs=['bold']) + CL_SQUARE_BRACKET
MESSAGES_TO_RETRY = [
"DB::Exception: ZooKeeper session has been expired",
"Coordination::Exception: Connection loss",
@ -51,7 +43,7 @@ def remove_control_characters(s):
def run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file):
if ext == '.sql':
command = "{0} --send_logs_level={1} --testmode --multiquery < {2} > {3} 2> {4}".format(args.client, server_logs_level, case_file, stdout_file, stderr_file)
command = "{0} --send_logs_level={1} --testmode --multiquery < {2} > {3} 2> {4}".format(args.client_with_database, server_logs_level, case_file, stdout_file, stderr_file)
else:
command = "{} > {} 2> {}".format(case_file, stdout_file, stderr_file)
@ -99,6 +91,20 @@ def main(args):
SERVER_DIED = False
def colored(text, color=None, on_color=None, attrs=None):
if sys.stdout.isatty() or args.force_color:
return termcolor.colored(text, color, on_color, attrs)
else:
return text
OP_SQUARE_BRACKET = colored("[", attrs=['bold'])
CL_SQUARE_BRACKET = colored("]", attrs=['bold'])
MSG_FAIL = OP_SQUARE_BRACKET + colored(" FAIL ", "red", attrs=['bold']) + CL_SQUARE_BRACKET
MSG_UNKNOWN = OP_SQUARE_BRACKET + colored(" UNKNOWN ", "yellow", attrs=['bold']) + CL_SQUARE_BRACKET
MSG_OK = OP_SQUARE_BRACKET + colored(" OK ", "green", attrs=['bold']) + CL_SQUARE_BRACKET
MSG_SKIPPED = OP_SQUARE_BRACKET + colored(" SKIPPED ", "cyan", attrs=['bold']) + CL_SQUARE_BRACKET
def is_data_present():
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, stderr) = clickhouse_proc.communicate("EXISTS TABLE test.hits")
@ -131,6 +137,7 @@ def main(args):
if args.configclient:
os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient)
os.environ.setdefault("CLICKHOUSE_TMP", tmp_dir)
os.environ.setdefault("CLICKHOUSE_DATABASE", args.database)
# Force to print server warnings in stderr
# Shell scripts could change logging level
@ -159,7 +166,12 @@ def main(args):
failures_total = 0
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test")
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database)
def is_test_from_dir(suite_dir, case):
case_file = os.path.join(suite_dir, case)
(name, ext) = os.path.splitext(case)
return os.path.isfile(case_file) and (ext == '.sql' or ext == '.sh' or ext == '.py')
def sute_key_func(item):
if args.order == 'random':
@ -190,7 +202,6 @@ def main(args):
suite = suite_re_obj.group(1)
if os.path.isdir(suite_dir):
print("\nRunning {} tests.\n".format(suite))
failures = 0
failures_chain = 0
@ -219,166 +230,180 @@ def main(args):
except ValueError:
return 99997
for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir)), key=key_func):
run_n, run_total = args.parallel.split('/')
run_n = float(run_n)
run_total = float(run_total)
all_tests = os.listdir(suite_dir)
all_tests = filter(lambda case: is_test_from_dir(suite_dir, case), all_tests)
all_tests = sorted(filter(lambda case: re.search(args.test, case) if args.test else True, all_tests), key=key_func)
tests_n = len(all_tests)
start = int(tests_n / run_total * (run_n - 1))
if start > 0:
start = start + 1
end = int(tests_n / run_total * (run_n))
all_tests = all_tests[start : end]
print("\nRunning {} {} tests.".format(tests_n, suite) + (" {} .. {} ".format(start, end) if run_total > 1 else "") + "\n")
for case in all_tests:
if SERVER_DIED:
break
case_file = os.path.join(suite_dir, case)
(name, ext) = os.path.splitext(case)
if os.path.isfile(case_file) and (ext == '.sql' or ext == '.sh' or ext == '.py'):
report_testcase = et.Element("testcase", attrib = {"name": name})
report_testcase = et.Element("testcase", attrib = {"name": name})
try:
print "{0:72}".format(name + ": "),
sys.stdout.flush()
try:
print "{0:72}".format(name + ": "),
sys.stdout.flush()
if args.skip and any(s in name for s in args.skip):
report_testcase.append(et.Element("skipped", attrib = {"message": "skip"}))
print(MSG_SKIPPED + " - skip")
skipped_total += 1
elif not args.zookeeper and 'zookeeper' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no zookeeper"}))
print(MSG_SKIPPED + " - no zookeeper")
skipped_total += 1
elif not args.shard and 'shard' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"}))
print(MSG_SKIPPED + " - no shard")
skipped_total += 1
elif not args.no_long and 'long' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no long"}))
print(MSG_SKIPPED + " - no long")
skipped_total += 1
if args.skip and any(s in name for s in args.skip):
report_testcase.append(et.Element("skipped", attrib = {"message": "skip"}))
print(MSG_SKIPPED + " - skip")
skipped_total += 1
elif not args.zookeeper and 'zookeeper' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no zookeeper"}))
print(MSG_SKIPPED + " - no zookeeper")
skipped_total += 1
elif not args.shard and 'shard' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"}))
print(MSG_SKIPPED + " - no shard")
skipped_total += 1
elif not args.no_long and 'long' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no long"}))
print(MSG_SKIPPED + " - no long")
skipped_total += 1
else:
disabled_file = os.path.join(suite_dir, name) + '.disabled'
if os.path.exists(disabled_file) and not args.disabled:
message = open(disabled_file, 'r').read()
report_testcase.append(et.Element("skipped", attrib = {"message": message}))
print(MSG_SKIPPED + " - " + message)
else:
disabled_file = os.path.join(suite_dir, name) + '.disabled'
if os.path.exists(disabled_file) and not args.disabled:
message = open(disabled_file, 'r').read()
report_testcase.append(et.Element("skipped", attrib = {"message": message}))
print(MSG_SKIPPED + " - " + message)
if args.testname:
clickhouse_proc = Popen(shlex.split(args.client_with_database), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite))
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file)
if proc.returncode is None:
try:
proc.kill()
except OSError as e:
if e.errno != ESRCH:
raise
failure = et.Element("failure", attrib = {"message": "Timeout"})
report_testcase.append(failure)
failures += 1
print("{0} - Timeout!".format(MSG_FAIL))
else:
counter = 1
while proc.returncode != 0 and need_retry(stderr):
proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file)
sleep(2**counter)
counter += 1
if counter > 6:
break
if args.testname:
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite))
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file)
if proc.returncode is None:
try:
proc.kill()
except OSError as e:
if e.errno != ESRCH:
raise
failure = et.Element("failure", attrib = {"message": "Timeout"})
if proc.returncode != 0:
failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures += 1
print("{0} - Timeout!".format(MSG_FAIL))
else:
counter = 1
while proc.returncode != 0 and need_retry(stderr):
proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file)
sleep(2**counter)
counter += 1
if counter > 6:
break
if proc.returncode != 0:
failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures += 1
failures_chain += 1
print("{0} - return code {1}".format(MSG_FAIL, proc.returncode))
if stderr:
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
print(stderr.encode('utf-8'))
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True
elif stderr:
failure = et.Element("failure", attrib = {"message": "having stderror"})
report_testcase.append(failure)
failures_chain += 1
print("{0} - return code {1}".format(MSG_FAIL, proc.returncode))
if stderr:
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
print(stderr.encode('utf-8'))
failures += 1
failures_chain += 1
print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8')))
elif 'Exception' in stdout:
failure = et.Element("error", attrib = {"message": "having exception"})
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True
elif stderr:
failure = et.Element("failure", attrib = {"message": "having stderror"})
report_testcase.append(failure)
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
failures += 1
failures_chain += 1
print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8')))
elif 'Exception' in stdout:
failure = et.Element("error", attrib = {"message": "having exception"})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures += 1
failures_chain += 1
print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8')))
elif not os.path.isfile(reference_file):
skipped = et.Element("skipped", attrib = {"message": "no reference file"})
report_testcase.append(skipped)
print("{0} - no reference file".format(MSG_UNKNOWN))
else:
result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE)
if result_is_different:
diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0]
diff = unicode(diff, errors='replace', encoding='utf-8')
cat = Popen(['cat', '-vet'], stdin=PIPE, stdout=PIPE).communicate(input=diff.encode(encoding='utf-8', errors='replace'))[0]
failure = et.Element("failure", attrib = {"message": "result differs with reference"})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
try:
stdout_element.text = et.CDATA(diff)
except:
stdout_element.text = et.CDATA(remove_control_characters(diff))
report_testcase.append(stdout_element)
failures += 1
failures_chain += 1
print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8')))
elif not os.path.isfile(reference_file):
skipped = et.Element("skipped", attrib = {"message": "no reference file"})
report_testcase.append(skipped)
print("{0} - no reference file".format(MSG_UNKNOWN))
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, cat.encode('utf-8')))
else:
result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE)
passed_total += 1
failures_chain = 0
print(MSG_OK)
if os.path.exists(stdout_file):
os.remove(stdout_file)
if os.path.exists(stderr_file):
os.remove(stderr_file)
except KeyboardInterrupt as e:
print(colored("Break tests execution", "red"))
raise e
except:
import traceback
exc_type, exc_value, tb = sys.exc_info()
error = et.Element("error", attrib = {"type": exc_type.__name__, "message": str(exc_value)})
report_testcase.append(error)
if result_is_different:
diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0]
diff = unicode(diff, errors='replace', encoding='utf-8')
cat = Popen(['cat', '-vet'], stdin=PIPE, stdout=PIPE).communicate(input=diff.encode(encoding='utf-8', errors='replace'))[0]
failures += 1
print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10))))
finally:
dump_report(args.output, suite, name, report_testcase)
failure = et.Element("failure", attrib = {"message": "result differs with reference"})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
try:
stdout_element.text = et.CDATA(diff)
except:
stdout_element.text = et.CDATA(remove_control_characters(diff))
report_testcase.append(stdout_element)
failures += 1
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, cat.encode('utf-8')))
else:
passed_total += 1
failures_chain = 0
print(MSG_OK)
if os.path.exists(stdout_file):
os.remove(stdout_file)
if os.path.exists(stderr_file):
os.remove(stderr_file)
except KeyboardInterrupt as e:
print(colored("Break tests execution", "red"))
raise e
except:
import traceback
exc_type, exc_value, tb = sys.exc_info()
error = et.Element("error", attrib = {"type": exc_type.__name__, "message": str(exc_value)})
report_testcase.append(error)
failures += 1
print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10))))
finally:
dump_report(args.output, suite, name, report_testcase)
if failures_chain >= 20:
break
if failures_chain >= 20:
break
failures_total = failures_total + failures
@ -390,7 +415,7 @@ def main(args):
print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total), "green", attrs=["bold"]))
if args.hung_check:
processlist = get_processlist(args.client)
processlist = get_processlist(args.client_with_database)
if processlist:
server_pid = get_server_pid(os.getenv("CLICKHOUSE_PORT_TCP", '9000'))
print(colored("\nFound hung queries in processlist:", "red", attrs=["bold"]))
@ -433,6 +458,9 @@ if __name__ == '__main__':
parser.add_argument('--order', default='desc', help='Run order (asc, desc, random)')
parser.add_argument('--testname', action='store_true', default=None, dest='testname', help='Make query with test name before test run')
parser.add_argument('--hung-check', action='store_true', default=False)
parser.add_argument('--force-color', action='store_true', default=False)
parser.add_argument('--database', default='test', help='Default database for tests')
parser.add_argument('--parallel', default='1/1', help='Parralel test run number/total')
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')
parser.add_argument('--skip', nargs='+', help="Skip these tests")
@ -473,6 +501,12 @@ if __name__ == '__main__':
args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST")
if os.getenv("CLICKHOUSE_PORT_TCP"):
args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP")
if os.getenv("CLICKHOUSE_DATABASE"):
args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE")
args.client_with_database = args.client
if args.database:
args.client_with_database += ' --database=' + args.database
if args.extract_from_config is None:
if os.access(args.binary + '-extract-from-config', os.X_OK):

View File

@ -131,13 +131,13 @@ else
TEST_DICT=${TEST_DICT=1}
CLICKHOUSE_CLIENT_QUERY="${CLICKHOUSE_CLIENT} --config ${CLICKHOUSE_CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -m -n -q"
$CLICKHOUSE_CLIENT_QUERY 'SELECT * from system.build_options; SELECT * FROM system.clusters;'
CLICKHOUSE_TEST="env ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}${CLICKHOUSE_BINARY_NAME} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
CLICKHOUSE_TEST="env ${TEST_DIR}clickhouse-test --force-color --binary ${BIN_DIR}${CLICKHOUSE_BINARY_NAME} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
CLICKHOUSE_PERFORMANCE_TEST="${BIN_DIR}clickhouse-performance-test --port $CLICKHOUSE_PORT_TCP --recursive $CUR_DIR/performance --skip-tags=long"
if [ "${TEST_RUN_STRESS}" ]; then
# Running test in parallel will fail some results (tests can create/fill/drop same tables)
TEST_NPROC=${TEST_NPROC:=$(( `nproc || sysctl -n hw.ncpu || echo 2` * 2))}
for i in `seq 1 ${TEST_NPROC}`; do
$CLICKHOUSE_TEST --order=random --testname &
$CLICKHOUSE_TEST --order=random --testname --tmp=$DATA_DIR/tmp/tmp${i} &
done
$CLICKHOUSE_PERFORMANCE_TEST &
fi

View File

@ -14,9 +14,9 @@ Don't use Docker from your system repository.
* [pip](https://pypi.python.org/pypi/pip). To install: `sudo apt-get install python-pip`
* [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka`
If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login.
(You must close all your sessions (for example, restart your computer))

View File

@ -384,6 +384,14 @@ def test_create_as_select(started_cluster):
ddl_check_query(instance, "DROP TABLE IF EXISTS test_as_select ON CLUSTER cluster")
def test_create_reserved(started_cluster):
instance = cluster.instances['ch2']
ddl_check_query(instance, "CREATE TABLE test_reserved ON CLUSTER cluster (`p` Date, `image` Nullable(String), `index` Nullable(Float64), `invalidate` Nullable(Int64)) ENGINE = MergeTree(`p`, `p`, 8192)")
ddl_check_query(instance, "CREATE TABLE test_as_reserved ON CLUSTER cluster ENGINE = Memory AS (SELECT * from test_reserved)")
ddl_check_query(instance, "DROP TABLE IF EXISTS test_reserved ON CLUSTER cluster")
ddl_check_query(instance, "DROP TABLE IF EXISTS test_as_reserved ON CLUSTER cluster")
if __name__ == '__main__':
with contextmanager(started_cluster)() as cluster:
for name, instance in cluster.instances.items():

View File

@ -6,7 +6,7 @@ CREATE TABLE default.test_table (EventDate Date, CounterID UInt32, UserID UInt6
CREATE MATERIALIZED VIEW default.test_view (Rows UInt64, MaxHitTime DateTime) ENGINE = Memory AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM default.test_table;
CREATE MATERIALIZED VIEW default.test_view_filtered (EventDate Date, CounterID UInt32) ENGINE = Memory POPULATE AS SELECT CounterID, EventDate FROM default.test_table WHERE EventDate < '2013-01-01';
INSERT INTO test_table (EventDate, UTCEventTime) VALUES ('2014-01-02', '2014-01-02 03:04:06');
INSERT INTO default.test_table (EventDate, UTCEventTime) VALUES ('2014-01-02', '2014-01-02 03:04:06');
SELECT * FROM default.test_table;
SELECT * FROM default.test_view;

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS numbers_memory;
CREATE TABLE numbers_memory AS system.numbers ENGINE = Memory;
INSERT INTO numbers_memory SELECT number FROM system.numbers LIMIT 100;
SELECT DISTINCT number FROM remote('127.0.0.{2,3}', default.numbers_memory) ORDER BY number LIMIT 10;
DROP TABLE numbers_memory;
DROP TABLE IF EXISTS test.numbers_memory;
CREATE TABLE test.numbers_memory AS system.numbers ENGINE = Memory;
INSERT INTO test.numbers_memory SELECT number FROM system.numbers LIMIT 100;
SELECT DISTINCT number FROM remote('127.0.0.{2,3}', test.numbers_memory) ORDER BY number LIMIT 10;
DROP TABLE test.numbers_memory;

View File

@ -48,9 +48,20 @@
7
8
9
0
1
2
3
4
5
6
7
8
9
< Content-Encoding: gzip
< Content-Encoding: deflate
< Content-Encoding: gzip
< Content-Encoding: br
1
1
1

View File

@ -12,12 +12,14 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=0" -H 'Accept-
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d;
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip, deflate' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d;
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: zip, eflate' -d 'SELECT number FROM system.numbers LIMIT 10';
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: br' -d 'SELECT number FROM system.numbers LIMIT 10' | brotli -d;
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding';
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding';
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: deflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding';
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: gzip, deflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding';
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: zip, eflate' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding';
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?enable_http_compression=1" -H 'Accept-Encoding: br' -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep --text '< Content-Encoding';
echo "SELECT 1" | ${CLICKHOUSE_CURL} -sS --data-binary @- ${CLICKHOUSE_URL};
echo "SELECT 1" | gzip -c | ${CLICKHOUSE_CURL} -sS --data-binary @- -H 'Content-Encoding: gzip' ${CLICKHOUSE_URL};

View File

@ -65,7 +65,7 @@ DROP TABLE IF EXISTS test.merge_one_two;
CREATE TABLE test.one (x Int32) ENGINE = Memory;
CREATE TABLE test.two (x UInt64) ENGINE = Memory;
CREATE TABLE test.merge_one_two (x UInt64) ENGINE = Merge(test, '^one|two$');
CREATE TABLE test.merge_one_two (x UInt64) ENGINE = Merge(test, '^one$|^two$');
INSERT INTO test.one VALUES (1);
INSERT INTO test.two VALUES (1);
@ -86,7 +86,7 @@ DROP TABLE IF EXISTS test.merge_one_two;
CREATE TABLE test.one (x String) ENGINE = Memory;
CREATE TABLE test.two (x FixedString(16)) ENGINE = Memory;
CREATE TABLE test.merge_one_two (x String) ENGINE = Merge(test, '^one|two$');
CREATE TABLE test.merge_one_two (x String) ENGINE = Merge(test, '^one$|^two$');
INSERT INTO test.one VALUES ('1');
INSERT INTO test.two VALUES ('1');
@ -102,7 +102,7 @@ DROP TABLE IF EXISTS test.merge_one_two;
CREATE TABLE test.one (x DateTime) ENGINE = Memory;
CREATE TABLE test.two (x UInt64) ENGINE = Memory;
CREATE TABLE test.merge_one_two (x UInt64) ENGINE = Merge(test, '^one|two$');
CREATE TABLE test.merge_one_two (x UInt64) ENGINE = Merge(test, '^one$|^two$');
INSERT INTO test.one VALUES (1);
INSERT INTO test.two VALUES (1);
@ -118,7 +118,7 @@ DROP TABLE IF EXISTS test.merge_one_two;
CREATE TABLE test.one (x Array(UInt32), z String DEFAULT '', y Array(UInt32)) ENGINE = Memory;
CREATE TABLE test.two (x Array(UInt64), z String DEFAULT '', y Array(UInt64)) ENGINE = Memory;
CREATE TABLE test.merge_one_two (x Array(UInt64), z String, y Array(UInt64)) ENGINE = Merge(test, '^one|two$');
CREATE TABLE test.merge_one_two (x Array(UInt64), z String, y Array(UInt64)) ENGINE = Merge(test, '^one$|^two$');
INSERT INTO test.one (x, y) VALUES ([1], [0]);
INSERT INTO test.two (x, y) VALUES ([1], [0]);

View File

@ -10,3 +10,4 @@ insert into test.table select today() as date, [number], [number + 1], toFixedSt
set preferred_max_column_in_block_size_bytes = 112;
select blockSize(), * from test.table prewhere x = 7 format Null;
drop table if exists test.table;

View File

@ -3,3 +3,4 @@ create table one_table (date Date, one UInt64) engine = MergeTree(date, (date, o
insert into one_table select today(), toUInt64(1) from system.numbers limit 100000;
SET preferred_block_size_bytes = 8192;
select isNull(one) from one_table where isNull(one);
drop table if exists one_table;

View File

@ -366,3 +366,5 @@ select 'arrayPopFront(arr1)';
select arr1, arrayPopFront(arr1) from test.array_functions;
select 'arrayPopBack(arr1)';
select arr1, arrayPopBack(arr1) from test.array_functions;
DROP TABLE if exists test.array_functions;

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
( ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}?query=SELECT%201";
${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}?query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "Query-Id:"
${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}?query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "X-ClickHouse-Query-Id:"
if [[ `${CLICKHOUSE_CURL} -sS -X POST -I "${CLICKHOUSE_URL}?query=SELECT+1" | grep -c '411 Length Required'` -ne 1 ]]; then
echo FAIL

View File

@ -14,4 +14,4 @@ desc table (select 1);
select '-';
desc (select * from system.numbers);
select '-';
drop table if exists test.tab;

View File

@ -10,3 +10,4 @@ insert into test.table select number, number / 8192 from system.numbers limit 10
alter table test.table add column def UInt64;
select * from test.table prewhere val > 2 format Null;
drop table if exists test.table;

View File

@ -1,15 +1,19 @@
drop table if exists test_in_tuple_1;
drop table if exists test_in_tuple_2;
drop table if exists test_in_tuple;
drop table if exists test.test_in_tuple_1;
drop table if exists test.test_in_tuple_2;
drop table if exists test.test_in_tuple;
create table test_in_tuple_1 (key Int32, key_2 Int32, x Array(Int32), y Array(Int32)) engine = MergeTree order by (key, key_2);
create table test_in_tuple_2 (key Int32, key_2 Int32, x Array(Int32), y Array(Int32)) engine = MergeTree order by (key, key_2);
create table test_in_tuple as test_in_tuple_1 engine = Merge('default', '^test_in_tuple_[0-9]+$');
create table test.test_in_tuple_1 (key Int32, key_2 Int32, x Array(Int32), y Array(Int32)) engine = MergeTree order by (key, key_2);
create table test.test_in_tuple_2 (key Int32, key_2 Int32, x Array(Int32), y Array(Int32)) engine = MergeTree order by (key, key_2);
create table test.test_in_tuple as test.test_in_tuple_1 engine = Merge('test', '^test_in_tuple_[0-9]+$');
insert into test_in_tuple_1 values (1, 1, [1, 2], [1, 2]);
insert into test_in_tuple_2 values (2, 1, [1, 2], [1, 2]);
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y order by _table;
insert into test.test_in_tuple_1 values (1, 1, [1, 2], [1, 2]);
insert into test.test_in_tuple_2 values (2, 1, [1, 2], [1, 2]);
select key, arr_x, arr_y, _table from test.test_in_tuple left array join x as arr_x, y as arr_y order by _table;
select '-';
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
select key, arr_x, arr_y, _table from test.test_in_tuple left array join x as arr_x, y as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
select '-';
select key, arr_x, arr_y, _table from test_in_tuple left array join arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), x, x ,y) as arr_x, arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), y, x ,y) as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
select key, arr_x, arr_y, _table from test.test_in_tuple left array join arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), x, x ,y) as arr_x, arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), y, x ,y) as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
drop table if exists test.test_in_tuple_1;
drop table if exists test.test_in_tuple_2;
drop table if exists test.test_in_tuple;

View File

@ -18,3 +18,4 @@ select arrayEnumerateUniq(arrayMap((a, b) -> (a, b), n.x, n.y)) from test.tab;
select arrayEnumerateUniq(arrayMap((a, b) -> (a, b), n.x, n.y), n.x) from test.tab;
select arrayEnumerateUniq(arrayMap((a, b) -> (a, b), n.x, n.y), arrayMap((a, b) -> (b, a), n.x, n.y)) from test.tab;
drop table test.tab;

View File

@ -1 +1 @@
Query-Id
X-ClickHouse-Query-Id

View File

@ -4,4 +4,4 @@ set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL_COMMAND} -I -sSg ${CLICKHOUSE_URL}?query=SELECT%201 | grep -o Query-Id
${CLICKHOUSE_CURL_COMMAND} -I -sSg ${CLICKHOUSE_URL}?query=SELECT%201 | grep -o X-ClickHouse-Query-Id

View File

@ -0,0 +1,12 @@
1 0
2 0
3 0
4 0
5 0
6 0
1 0
2 0
3 0
4 0
5 0
6 0

View File

@ -0,0 +1,16 @@
set allow_experimental_multiple_joins_emulation = 0;
set allow_experimental_cross_to_join_conversion = 0;
select ax, c from (select [1,2] ax, 0 c) array join ax join (select 0 c) using(c);
select ax, c from (select [3,4] ax, 0 c) join (select 0 c) using(c) array join ax;
select ax, c from (select [5,6] ax, 0 c) s1 join system.one s2 ON s1.c = s2.dummy array join ax;
set allow_experimental_multiple_joins_emulation = 1;
set allow_experimental_cross_to_join_conversion = 1;
select ax, c from (select [1,2] ax, 0 c) array join ax join (select 0 c) using(c);
select ax, c from (select [3,4] ax, 0 c) join (select 0 c) using(c) array join ax;
select ax, c from (select [5,6] ax, 0 c) s1 join system.one s2 ON s1.c = s2.dummy array join ax;
select ax, c from (select [7,8] ax, 0 c) s1
join system.one s2 ON s1.c = s2.dummy
join system.one s3 ON s1.c = s3.dummy
array join ax; -- { serverError 48 }

View File

@ -0,0 +1,3 @@
0
0
0

View File

@ -0,0 +1,13 @@
USE test;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (n Int32) ENGINE = Memory;
CREATE TABLE t2 (a Int32, n Int32) ENGINE = Memory;
SELECT count() FROM t1 WHERE if(1, 1, n = 0);
SELECT count(n) FROM t2 WHERE if(1, 1, n = 0);
SELECT count() FROM t2 WHERE if(1, 1, n = 0);
DROP TABLE t1;
DROP TABLE t2;

View File

@ -0,0 +1,12 @@
One Hundred
Two Hundred
One Hundred
Two Hundred
One Hundred
Two Hundred
One Hundred
Two Hundred
One Hundred
Two Hundred
One Hundred
Two Hundred

View File

@ -0,0 +1,59 @@
USE test;
DROP TABLE IF EXISTS local_table;
DROP TABLE IF EXISTS other_table;
CREATE TABLE local_table
(
id Int32,
name String,
ts DateTime,
oth_id Int32
) ENGINE = MergeTree() PARTITION BY toMonday(ts) ORDER BY (ts, id);
CREATE TABLE other_table
(
id Int32,
name String,
ts DateTime,
trd_id Int32
) ENGINE = MergeTree() PARTITION BY toMonday(ts) ORDER BY (ts, id);
INSERT INTO local_table VALUES(1, 'One', now(), 100);
INSERT INTO local_table VALUES(2, 'Two', now(), 200);
INSERT INTO other_table VALUES(100, 'One Hundred', now(), 1000);
INSERT INTO other_table VALUES(200, 'Two Hundred', now(), 2000);
-- FIXME: test.other_table -> other_table in first query breaks test (external tables cache error)
select t2.name from remote('127.0.0.2', 'test.local_table') as t1
left join test.other_table as t2
on t1.oth_id = t2.id
order by t2.name;
select t2.name from test.other_table as t2
global right join remote('127.0.0.2', 'test.local_table') as t1
on t1.oth_id = t2.id
order by t2.name;
select t2.name from remote('127.0.0.2', 'test.local_table') as t1
global left join other_table as t2
on t1.oth_id = t2.id
order by t2.name;
select t2.name from remote('127.0.0.2', 'test.local_table') as t1
global left join other_table as t2
on t1.oth_id = t2.id
order by t2.name;
select other_table.name from remote('127.0.0.2', 'test.local_table') as t1
global left join other_table
on t1.oth_id = other_table.id
order by other_table.name;
select other_table.name from remote('127.0.0.2', 'test.local_table') as t1
global left join other_table as t2
on t1.oth_id = other_table.id
order by other_table.name;
DROP TABLE local_table;
DROP TABLE other_table;

View File

@ -6,24 +6,24 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.cannot_kill_query"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test.cannot_kill_query (x UInt64) ENGINE = MergeTree ORDER BY x" &> /dev/null
$CLICKHOUSE_CLIENT -q "INSERT INTO test.cannot_kill_query SELECT * FROM numbers(10000000)" &> /dev/null
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS cannot_kill_query"
$CLICKHOUSE_CLIENT -q "CREATE TABLE cannot_kill_query (x UInt64) ENGINE = MergeTree ORDER BY x" &> /dev/null
$CLICKHOUSE_CLIENT -q "INSERT INTO cannot_kill_query SELECT * FROM numbers(10000000)" &> /dev/null
# This SELECT query will run for a long time. It's used as bloker for ALTER query. It will be killed with SYNC kill.
query_for_pending="SELECT count() FROM test.cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1, max_block_size=1"
query_for_pending="SELECT count() FROM cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1, max_block_size=1"
$CLICKHOUSE_CLIENT -q "$query_for_pending" &>/dev/null &
sleep 1 # queries should be in strict order
# This ALTER query will wait until $query_for_pending finished. Also it will block $query_to_kill.
$CLICKHOUSE_CLIENT -q "ALTER TABLE test.cannot_kill_query MODIFY COLUMN x UInt64" &>/dev/null &
$CLICKHOUSE_CLIENT -q "ALTER TABLE cannot_kill_query MODIFY COLUMN x UInt64" &>/dev/null &
sleep 1
# This SELECT query will also run for a long time. Also it's blocked by ALTER query. It will be killed with ASYNC kill.
# This is main idea which we check -- blocked queries can be killed with ASYNC kill.
query_to_kill="SELECT sum(1) FROM test.cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1"
query_to_kill="SELECT sum(1) FROM cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1"
$CLICKHOUSE_CLIENT -q "$query_to_kill" &>/dev/null &
sleep 1 # just to be sure that kill of $query_to_kill will be executed after $query_to_kill.
@ -48,4 +48,4 @@ do
fi
done
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.cannot_kill_query" &>/dev/null
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS cannot_kill_query" &>/dev/null

View File

@ -1,31 +0,0 @@
USE test;
DROP TABLE IF EXISTS tvs;
DROP TABLE IF EXISTS trades;
DROP TABLE IF EXISTS keys;
DROP TABLE IF EXISTS tv_times;
DROP TABLE IF EXISTS trade_times;
CREATE TABLE keys(k UInt32) ENGINE = MergeTree() ORDER BY k;
INSERT INTO keys(k) SELECT number FROM system.numbers LIMIT 5000;
CREATE TABLE tv_times(t UInt32) ENGINE = MergeTree() ORDER BY t;
INSERT INTO tv_times(t) SELECT number * 3 FROM system.numbers LIMIT 50000;
CREATE TABLE trade_times(t UInt32) ENGINE = MergeTree() ORDER BY t;
INSERT INTO trade_times(t) SELECT number * 10 FROM system.numbers LIMIT 15000;
CREATE TABLE tvs(k UInt32, t UInt32, tv UInt64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO tvs(k,t,tv) SELECT k, t, t FROM keys CROSS JOIN tv_times;
CREATE TABLE trades(k UInt32, t UInt32, price UInt64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO trades(k,t,price) SELECT k, t, t FROM keys CROSS JOIN trade_times;
SELECT SUM(trades.price - tvs.tv) FROM trades ASOF LEFT JOIN tvs USING(k,t);
DROP TABLE tvs;
DROP TABLE trades;
DROP TABLE keys;
DROP TABLE tv_times;
DROP TABLE trade_times;

View File

@ -0,0 +1 @@
3000000

View File

@ -0,0 +1,16 @@
USE test;
DROP TABLE IF EXISTS tvs;
CREATE TABLE tvs(k UInt32, t UInt32, tv UInt64) ENGINE = Memory;
INSERT INTO tvs(k,t,tv) SELECT k, t, t
FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys
CROSS JOIN (SELECT toUInt32(number * 3) as t FROM numbers(10000)) tv_times;
SELECT SUM(trades.price - tvs.tv) FROM
(SELECT k, t, t as price
FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys
CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times) trades
ASOF LEFT JOIN tvs USING(k,t);
DROP TABLE tvs;

View File

@ -5,8 +5,6 @@ set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "USE test;"
for typename in "UInt32" "UInt64" "Float64" "Float32"
do
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS A;"
@ -19,4 +17,7 @@ do
$CLICKHOUSE_CLIENT -q "INSERT INTO B(k,t,b) VALUES (2,3,3);"
$CLICKHOUSE_CLIENT -q "SELECT k, t, a, b FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (k,t);"
$CLICKHOUSE_CLIENT -q "DROP TABLE A;"
$CLICKHOUSE_CLIENT -q "DROP TABLE B;"
done

View File

@ -4,13 +4,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.small_table"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a);"
$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a);"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.small_table(n) SELECT * from system.numbers limit 100000;"
$CLICKHOUSE_CLIENT --query="INSERT INTO small_table(n) SELECT * from system.numbers limit 100000;"
cached_query="SELECT count() FROM test.small_table where n > 0;"
cached_query="SELECT count() FROM small_table where n > 0;"
$CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query="$cached_query" &> /dev/null
@ -21,5 +21,5 @@ $CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 2) ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.small_table"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"

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