diff --git a/CMakeLists.txt b/CMakeLists.txt index 6ac4d67f6ae..f84a181a39c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -264,11 +264,10 @@ if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") endif () -# TODO: only make this extra-checks in CI builds, since a lot of contrib libs won't link - -# CI works around this problem by explicitly adding GLIBC_COMPATIBILITY flag. -if (NOT SANITIZE AND YANDEX_OFFICIAL_BUILD) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") +# Make this extra-checks for correct library dependencies. +if (NOT SANITIZE) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") endif () include (cmake/find_unwind.cmake) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index bc229deeced..843ff9cd8af 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -219,7 +219,7 @@ endif() add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) add_dependencies(${ARROW_LIBRARY} protoc) target_include_directories(${ARROW_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src ${Boost_INCLUDE_DIRS}) -target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${PROTOBUF_LIBRARIES} Threads::Threads) +target_link_libraries(${ARROW_LIBRARY} PRIVATE ${DOUBLE_CONVERSION_LIBRARIES} ${Protobuf_LIBRARY} Threads::Threads) if (ARROW_WITH_LZ4) target_link_libraries(${ARROW_LIBRARY} PRIVATE ${LZ4_LIBRARY}) endif() diff --git a/dbms/src/AggregateFunctions/QuantileExact.h b/dbms/src/AggregateFunctions/QuantileExact.h index 5a1343b1399..4a2aa574ae9 100644 --- a/dbms/src/AggregateFunctions/QuantileExact.h +++ b/dbms/src/AggregateFunctions/QuantileExact.h @@ -176,7 +176,7 @@ struct QuantileExactExclusive : public QuantileExact } }; -/// QuantileExactInclusive is equivalent to Excel PERCENTILE and PERCENTILE.INC, R-7, SciPy-(1,1) +/// QuantileExactInclusive is equivalent to Excel PERCENTILE and PERCENTILE.INC, R-7, SciPy-(1,1) template struct QuantileExactInclusive : public QuantileExact { diff --git a/dbms/src/Common/OptimizedRegularExpression.cpp b/dbms/src/Common/OptimizedRegularExpression.cpp index c87d87fc2df..3a224709447 100644 --- a/dbms/src/Common/OptimizedRegularExpression.cpp +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -1,4 +1,5 @@ #include +#include #include #define MIN_LENGTH_FOR_STRSTR 3 @@ -413,9 +414,9 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject return 0; } - StringPieceType pieces[MAX_SUBPATTERNS]; + DB::PODArrayWithStackMemory pieces(limit); - if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces, limit)) + if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces.data(), pieces.size())) return 0; else { diff --git a/dbms/src/Common/StringSearcher.h b/dbms/src/Common/StringSearcher.h index 5e78ff23df1..25287db11f5 100644 --- a/dbms/src/Common/StringSearcher.h +++ b/dbms/src/Common/StringSearcher.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include #include @@ -23,6 +25,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_PARAMETER; + extern const int BAD_ARGUMENTS; } @@ -157,7 +160,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { static const Poco::UTF8Encoding utf8; @@ -374,7 +377,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { #ifdef __SSE4_1__ if (pageSafe(pos)) @@ -567,7 +570,7 @@ public: #endif } - ALWAYS_INLINE bool compare(const UInt8 * pos) const + ALWAYS_INLINE bool compare(const UInt8 * /*haystack*/, const UInt8 * /*haystack_end*/, const UInt8 * pos) const { #ifdef __SSE4_1__ if (pageSafe(pos)) @@ -697,11 +700,81 @@ public: } }; +// Searches for needle surrounded by token-separators. +// Separators are anything inside ASCII (0-128) and not alphanum. +// Any value outside of basic ASCII (>=128) is considered a non-separator symbol, hence UTF-8 strings +// should work just fine. But any Unicode whitespace is not considered a token separtor. +template +class TokenSearcher +{ + StringSearcher searcher; + size_t needle_size; + +public: + TokenSearcher(const char * const needle_, const size_t needle_size_) + : searcher{needle_, needle_size_}, + needle_size(needle_size_) + { + if (std::any_of(reinterpret_cast(needle_), reinterpret_cast(needle_) + needle_size_, isTokenSeparator)) + { + throw Exception{"Needle must not contain whitespace or separator characters", ErrorCodes::BAD_ARGUMENTS}; + } + + } + + ALWAYS_INLINE bool compare(const UInt8 * haystack, const UInt8 * haystack_end, const UInt8 * pos) const + { + // use searcher only if pos is in the beginning of token and pos + searcher.needle_size is end of token. + if (isToken(haystack, haystack_end, pos)) + return searcher.compare(haystack, haystack_end, pos); + + return false; + } + + const UInt8 * search(const UInt8 * haystack, const UInt8 * const haystack_end) const + { + // use searcher.search(), then verify that returned value is a token + // if it is not, skip it and re-run + + const UInt8 * pos = haystack; + while (pos < haystack_end) + { + pos = searcher.search(pos, haystack_end); + if (pos == haystack_end || isToken(haystack, haystack_end, pos)) + return pos; + + // assuming that heendle does not contain any token separators. + pos += needle_size; + } + return haystack_end; + } + + const UInt8 * search(const UInt8 * haystack, const size_t haystack_size) const + { + return search(haystack, haystack + haystack_size); + } + + ALWAYS_INLINE bool isToken(const UInt8 * haystack, const UInt8 * const haystack_end, const UInt8* p) const + { + return (p == haystack || isTokenSeparator(*(p - 1))) + && (p + needle_size >= haystack_end || isTokenSeparator(*(p + needle_size))); + } + + ALWAYS_INLINE static bool isTokenSeparator(const UInt8 c) + { + if (isAlphaNumericASCII(c) || !isASCII(c)) + return false; + + return true; + } +}; + using ASCIICaseSensitiveStringSearcher = StringSearcher; using ASCIICaseInsensitiveStringSearcher = StringSearcher; using UTF8CaseSensitiveStringSearcher = StringSearcher; using UTF8CaseInsensitiveStringSearcher = StringSearcher; +using ASCIICaseSensitiveTokenSearcher = TokenSearcher; /** Uses functions from libc. diff --git a/dbms/src/Common/Volnitsky.h b/dbms/src/Common/Volnitsky.h index 748cbe09138..c87bdd79dab 100644 --- a/dbms/src/Common/Volnitsky.h +++ b/dbms/src/Common/Volnitsky.h @@ -327,6 +327,8 @@ protected: FallbackSearcher fallback_searcher; public: + using Searcher = FallbackSearcher; + /** haystack_size_hint - the expected total size of the haystack for `search` calls. Optional (zero means unspecified). * If you specify it small enough, the fallback algorithm will be used, * since it is considered that it's useless to waste time initializing the hash table. @@ -373,7 +375,7 @@ public: const auto res = pos - (hash[cell_num] - 1); /// pointer in the code is always padded array so we can use pagesafe semantics - if (fallback_searcher.compare(res)) + if (fallback_searcher.compare(haystack, haystack_end, res)) return res; } } @@ -520,7 +522,7 @@ public: { const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) return true; } } @@ -552,7 +554,7 @@ public: { const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) ans = std::min(ans, ind); } } @@ -590,7 +592,7 @@ public: { const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) ans = std::min(ans, res - haystack); } } @@ -625,7 +627,7 @@ public: { const auto * res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (ans[ind] == 0 && res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res)) + if (ans[ind] == 0 && res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) ans[ind] = count_chars(haystack, res); } } @@ -650,6 +652,8 @@ using VolnitskyUTF8 = VolnitskyBase; /// ignores non-ASCII bytes using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase; +using VolnitskyToken = VolnitskyBase; + using MultiVolnitsky = MultiVolnitskyBase; using MultiVolnitskyUTF8 = MultiVolnitskyBase; using MultiVolnitskyCaseInsensitive = MultiVolnitskyBase; diff --git a/dbms/src/DataTypes/DataTypeEnum.cpp b/dbms/src/DataTypes/DataTypeEnum.cpp index cffc29feaf8..add7052195a 100644 --- a/dbms/src/DataTypes/DataTypeEnum.cpp +++ b/dbms/src/DataTypes/DataTypeEnum.cpp @@ -234,7 +234,7 @@ void DataTypeEnum::deserializeBinaryBulk( } template -void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const +void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { if (value_index) return; diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 726eb8738af..5d688232bd4 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -434,6 +434,74 @@ struct MultiSearchFirstIndexImpl } }; +/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. + */ +template +struct HasTokenImpl +{ + using ResultType = UInt8; + + static void vector_constant( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) + { + if (offsets.empty()) + return; + + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// The current index in the array of strings. + size_t i = 0; + + VolnitskyToken searcher(pattern.data(), pattern.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Let's determine which index it refers to. + while (begin + offsets[i] <= pos) + { + res[i] = negate_result; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + pattern.size() < begin + offsets[i]) + res[i] = !negate_result; + else + res[i] = negate_result; + + pos = begin + offsets[i]; + ++i; + } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0])); + } + + static void constant_constant(const std::string & data, const std::string & pattern, UInt8 & res) + { + VolnitskyToken searcher(pattern.data(), pattern.size(), data.size()); + const auto found = searcher.search(data.c_str(), data.size()) != data.end().base(); + res = negate_result ^ found; + } + + template + static void vector_vector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } + + /// Search different needles in single haystack. + template + static void constant_vector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + struct NamePosition { @@ -516,6 +584,11 @@ struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8"; }; +struct NameHasToken +{ + static constexpr auto name = "hasToken"; +}; + using FunctionPosition = FunctionsStringSearch, NamePosition>; using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; @@ -542,6 +615,7 @@ using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitiveUTF8>; +using FunctionHasToken = FunctionsStringSearch, NameHasToken>; void registerFunctionsStringSearch(FunctionFactory & factory) { @@ -570,6 +644,8 @@ void registerFunctionsStringSearch(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/Functions/GeoUtils.cpp b/dbms/src/Functions/GeoUtils.cpp index 5134343dae0..847d934c6b4 100644 --- a/dbms/src/Functions/GeoUtils.cpp +++ b/dbms/src/Functions/GeoUtils.cpp @@ -18,7 +18,7 @@ const UInt8 geohash_base32_decode_lookup_table[256] = { 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, - 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 10, 11, 12, 13, 14, 15, 16, 0xFF, 17, 18, 0xFF, 19, 20, 0xFF, diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index f2e2543cc90..46f69530005 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -1,10 +1,8 @@ #include #include #include +#include -#ifdef __SSE4_2__ -#include -#endif namespace DB { @@ -60,7 +58,7 @@ public: execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); res_data.resize(res_data.size() + length + 1); - memcpy(&res_data[res_offset], start, length); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], start, length); res_offset += length + 1; res_data[res_offset - 1] = '\0'; @@ -77,59 +75,27 @@ public: private: static void execute(const UInt8 * data, size_t size, const UInt8 *& res_data, size_t & res_size) { - size_t chars_to_trim_left = 0; - size_t chars_to_trim_right = 0; - char whitespace = ' '; -#ifdef __SSE4_2__ - const auto bytes_sse = sizeof(__m128i); - const auto size_sse = size - (size % bytes_sse); - const auto whitespace_mask = _mm_set1_epi8(whitespace); - constexpr auto base_sse_mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY; - auto mask = bytes_sse; -#endif + const char * char_data = reinterpret_cast(data); + const char * char_end = char_data + size; if constexpr (mode::trim_left) { -#ifdef __SSE4_2__ - /// skip whitespace from left in blocks of up to 16 characters - - /// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate - enum { left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT }; - while (mask == bytes_sse && chars_to_trim_left < size_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(data + chars_to_trim_left)); - mask = _mm_cmpistri(whitespace_mask, chars, left_sse_mode); - chars_to_trim_left += mask; - } -#endif - /// skip remaining whitespace from left, character by character - while (chars_to_trim_left < size && data[chars_to_trim_left] == whitespace) - ++chars_to_trim_left; + const char * found = find_first_not_symbols<' '>(char_data, char_end); + size_t num_chars = found - char_data; + char_data += num_chars; } if constexpr (mode::trim_right) { - const auto trim_right_size = size - chars_to_trim_left; -#ifdef __SSE4_2__ - /// try to skip whitespace from right in blocks of up to 16 characters - - /// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate - enum { right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT }; - const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse); - while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(data + size - chars_to_trim_right - bytes_sse)); - mask = _mm_cmpistri(whitespace_mask, chars, right_sse_mode); - chars_to_trim_right += mask; - } -#endif - /// skip remaining whitespace from right, character by character - while (chars_to_trim_right < trim_right_size && data[size - chars_to_trim_right - 1] == whitespace) - ++chars_to_trim_right; + const char * found = find_last_not_symbols_or_null<' '>(char_data, char_end); + if (found) + char_end = found + 1; + else + char_end = char_data; } - res_data = data + chars_to_trim_left; - res_size = size - chars_to_trim_left - chars_to_trim_right; + res_data = reinterpret_cast(char_data); + res_size = char_end - char_data; } }; diff --git a/dbms/src/IO/HDFSCommon.cpp b/dbms/src/IO/HDFSCommon.cpp index 0f1a58942d6..a94fbeabd60 100644 --- a/dbms/src/IO/HDFSCommon.cpp +++ b/dbms/src/IO/HDFSCommon.cpp @@ -40,7 +40,10 @@ HDFSBuilderPtr createHDFSBuilder(const Poco::URI & uri) hdfsBuilderSetUserName(builder.get(), user.c_str()); } hdfsBuilderSetNameNode(builder.get(), host.c_str()); - hdfsBuilderSetNameNodePort(builder.get(), port); + if (port != 0) + { + hdfsBuilderSetNameNodePort(builder.get(), port); + } return builder; } diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 25fecc5c642..1cd9d329c9d 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -12,7 +12,7 @@ namespace DB ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSchemaInfo & info_) : IRowInputFormat(header_, in_, params_) - , data_types(header_.getDataTypes()) + , data_types(header_.getDataTypes()) , reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info_), header_.getNames()) { } diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 3625c6f1aa5..246ad6784b2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -168,6 +168,19 @@ const MergeTreeConditionFullText::AtomMap MergeTreeConditionFullText::atom_map return true; } }, + { + "hasToken", + [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) + { + out.function = RPNElement::FUNCTION_EQUALS; + out.bloom_filter = std::make_unique( + idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed); + + const auto & str = value.get(); + stringToBloomFilter(str.c_str(), str.size(), idx.token_extractor_func, *out.bloom_filter); + return true; + } + }, { "startsWith", [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 40aba822353..954ac774583 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -405,25 +405,6 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) const return true; } -static bool checkAtomName(const String & name) -{ - static std::set atoms = { - "notEquals", - "equals", - "less", - "greater", - "lessOrEquals", - "greaterOrEquals", - "in", - "notIn", - "like", - "startsWith", - "endsWith", - "multiSearchAny" - }; - return atoms.find(name) != atoms.end(); -} - bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr &node, bool atomic) const { if (const auto * func = node->as()) @@ -439,16 +420,14 @@ bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr &node, bool atomic return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic); else if (func->name == "not") return checkASTUseless(args[0], atomic); - else if (!atomic && checkAtomName(func->name)) - return checkASTUseless(node, true); else return std::any_of(args.begin(), args.end(), - [this, &atomic](const auto & arg) { return checkASTUseless(arg, atomic); }); + [this](const auto & arg) { return checkASTUseless(arg, true); }); } else if (const auto * literal = node->as()) return !atomic && literal->value.get(); else if (const auto * identifier = node->as()) - return key_columns.find(identifier->getColumnName()) == key_columns.end(); + return key_columns.find(identifier->getColumnName()) == std::end(key_columns); else return true; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2aae847217e..932721eb028 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -108,7 +108,7 @@ size_t MergeTreeRangeReader::DelayedStream::finalize(Block & block) MergeTreeRangeReader::Stream::Stream( - size_t from_mark, size_t to_mark, MergeTreeReader * merge_tree_reader_) + size_t from_mark, size_t to_mark, MergeTreeReader * merge_tree_reader_) : current_mark(from_mark), offset_after_current_mark(0) , last_mark(to_mark) , merge_tree_reader(merge_tree_reader_) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 9552373901c..0eae69ee17e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -41,7 +41,7 @@ public: { public: DelayedStream() = default; - DelayedStream(size_t from_mark, MergeTreeReader * merge_tree_reader); + DelayedStream(size_t from_mark, MergeTreeReader * merge_tree_reader); /// Read @num_rows rows from @from_mark starting from @offset row /// Returns the number of rows added to block. diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index 2afe2a7c018..2f155e22a11 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -146,7 +146,7 @@ BlockInputStreams StorageSystemNumbers::read( res[i] = std::make_shared(max_block_size, offset + i * max_block_size, num_streams * max_block_size); if (limit) /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - res[i] = std::make_shared(res[i], *limit * (i + 1) / num_streams - *limit * i / num_streams, 0, false, true); + res[i] = std::make_shared(res[i], *limit * (i + 1) / num_streams - *limit * i / num_streams, 0, false, true); } return res; diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.reference b/dbms/tests/queries/0_stateless/00600_replace_running_query.reference index 804267a1c11..a01672aae85 100644 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.reference +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.reference @@ -2,6 +2,5 @@ 1 1 1 -finished 42 readonly SELECT 2, count() FROM system.numbers 1 44 diff --git a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh index 513f6d8440e..dbbf41dd772 100755 --- a/dbms/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/dbms/tests/queries/0_stateless/00600_replace_running_query.sh @@ -30,7 +30,7 @@ ${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 43' 2>&1 | grep -cF 'is alrea # Trying to replace query of a different user $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=42&replace_running_query=1" -d 'SELECT 1' | grep -cF 'is already running by user' -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" > /dev/null wait ${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & diff --git a/dbms/tests/queries/0_stateless/00942_mutate_index.sh b/dbms/tests/queries/0_stateless/00942_mutate_index.sh index c6dd1dfb836..467eb9ab671 100755 --- a/dbms/tests/queries/0_stateless/00942_mutate_index.sh +++ b/dbms/tests/queries/0_stateless/00942_mutate_index.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" @@ -35,8 +36,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" - -sleep 0.1 +wait_for_mutation "minmax_idx" "mutation_2.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index f51b66993aa..bc59b41b005 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" @@ -38,22 +39,26 @@ SET allow_experimental_data_skipping_indices=1; ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_3.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" +wait_for_mutation "minmax_idx" "mutation_4.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;" +sleep 0.5 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" +wait_for_mutation "minmax_idx" "mutation_5.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 9047bbb3a72..74f15e63545 100755 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" @@ -42,7 +43,7 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -sleep 0.5 +wait_for_mutation "minmax_idx" "mutation_3.txt" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh index 613226a3fb7..765dfb6abe5 100755 --- a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -2,6 +2,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions1;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;" @@ -44,19 +45,20 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES (9, 1, 2)" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" sleep 1 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" $CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" -sleep 1 +wait_for_mutation "indices_mutaions1" "0000000000" "test" +wait_for_mutation "indices_mutaions2" "0000000000" "test" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" $CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions1" $CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions2" diff --git a/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh b/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh index 64fae3fb0f9..14f2a8e31fb 100755 --- a/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh +++ b/dbms/tests/queries/0_stateless/00984_parser_stack_overflow.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +CLICKHOUSE_CURL_TIMEOUT=30 + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.python b/dbms/tests/queries/0_stateless/00990_hasToken.python new file mode 100755 index 00000000000..217d96dfe52 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken.python @@ -0,0 +1,124 @@ +#!/usr/bin/env python +# encoding: utf-8 + +import re + +HAYSTACKS = [ + "hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay needle", + "needle hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay", + "hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay", +] + +NEEDLE = "needle" + +HAY_RE = re.compile(r'\bhay\b', re.IGNORECASE) +NEEDLE_RE = re.compile(r'\bneedle\b', re.IGNORECASE) + +def replace_follow_case(replacement): + def func(match): + g = match.group() + if g.islower(): return replacement.lower() + if g.istitle(): return replacement.title() + if g.isupper(): return replacement.upper() + return replacement + return func + +def replace_separators(query, new_sep): + SEP_RE = re.compile('\\s+') + result = SEP_RE.sub(new_sep, query) + return result + +def enlarge_haystack(query, times, separator=''): + return HAY_RE.sub(replace_follow_case(('hay' + separator) * times), query) + +def small_needle(query): + return NEEDLE_RE.sub(replace_follow_case('n'), query) + +def remove_needle(query): + return NEEDLE_RE.sub('', query) + +def replace_needle(query, new_needle): + return NEEDLE_RE.sub(new_needle, query) + +# with str.lower, str.uppert, str.title and such +def transform_needle(query, string_transformation_func): + def replace_with_transformation(match): + g = match.group() + return string_transformation_func(g) + + return NEEDLE_RE.sub(replace_with_transformation, query) + + +def create_cases(table_row_template, table_query_template, const_query_template): + const_queries = [] + table_rows = [] + table_queries = set() + + def add_case(haystack, needle, match): + match = int(match) + const_queries.append(const_query_template.format(haystack=haystack, needle=needle, match=match)) + table_queries.add(table_query_template.format(haystack=haystack, needle=needle, match=match)) + table_rows.append(table_row_template.format(haystack=haystack, needle=needle, match=match)) + + # Negative cases + add_case(remove_needle(HAYSTACKS[0]), NEEDLE, False) + for haystack in HAYSTACKS: + add_case(transform_needle(haystack, str.title), NEEDLE, False) + sep = '' + h = replace_separators(haystack, sep) + add_case(h, NEEDLE, False) + add_case(small_needle(h), small_needle(NEEDLE), False) + add_case(enlarge_haystack(h, 10, sep), NEEDLE, False) + + # positive cases + for haystack in HAYSTACKS: + add_case(transform_needle(haystack, str.title), transform_needle(NEEDLE, str.title), True) + add_case(transform_needle(haystack, str.upper), transform_needle(NEEDLE, str.upper), True) + + # Not checking all separators since some (like ' and \n) cause issues when coupled with + # re-based replacement and quoting in query + # other are rare in practice and checking all separators makes this test too lengthy. + + # r'\\\\' turns into a single '\' in query + #separators = list(''' \t`~!@#$%^&*()-=+|]}[{";:/?.>,<''') + [r'\\\\'] + separators = list(''' \t;:?.,''') + [r'\\\\'] + for sep in separators: + h = replace_separators(haystack, sep) + add_case(h, NEEDLE, True) + add_case(small_needle(h), small_needle(NEEDLE), True) + add_case(enlarge_haystack(h, 200, sep), NEEDLE, True) + add_case(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) + add_case(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + + return table_rows, table_queries, const_queries + +def main(): + + def query(x): + print x + + CONST_QUERY = """SELECT hasToken('{haystack}', '{needle}'), ' expecting ', {match};""" + #SELECT hasToken(haystack, '{needle}') FROM ht WHERE needle = '{needle}' AND match = {match};""" + TABLE_QUERY = """WITH '{needle}' as n SELECT haystack, needle, hasToken(haystack, n) as result FROM ht WHERE needle = n AND result != match;""" + TABLE_ROW = """('{haystack}', '{needle}', {match})""" + + rows, table_queries, const_queries = create_cases(TABLE_ROW, TABLE_QUERY, CONST_QUERY) + for q in const_queries: + query(q) + + query("""DROP TABLE IF EXISTS ht; + CREATE TABLE IF NOT EXISTS + ht +( + haystack String, + needle String, + match UInt8 +) +ENGINE MergeTree() +ORDER BY haystack; +INSERT INTO ht VALUES {values};""".format(values=", ".join(rows))) + for q in sorted(table_queries): + query(q) + +if __name__ == '__main__': + main() diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.reference b/dbms/tests/queries/0_stateless/00990_hasToken.reference new file mode 100644 index 00000000000..867c0c1c691 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken.reference @@ -0,0 +1,139 @@ +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.sh b/dbms/tests/queries/0_stateless/00990_hasToken.sh new file mode 100755 index 00000000000..4ccb77b8ecc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test + +python $CURDIR/00990_hasToken.python | ${CLICKHOUSE_CLIENT} -nm diff --git a/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference new file mode 100644 index 00000000000..10e8f0d2c59 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.reference @@ -0,0 +1,3 @@ +2007 +2007 +2007 diff --git a/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql new file mode 100644 index 00000000000..60e4d959417 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql @@ -0,0 +1,33 @@ +SET allow_experimental_data_skipping_indices = 1; + +DROP TABLE IF EXISTS bloom_filter; + +CREATE TABLE bloom_filter +( + id UInt64, + s String, + INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1 +) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 8; + +insert into bloom_filter select number, 'yyy,uuu' from numbers(1024); +insert into bloom_filter select number+2000, 'abc,def,zzz' from numbers(8); +insert into bloom_filter select number+3000, 'yyy,uuu' from numbers(1024); +insert into bloom_filter select number+3000, 'abcdefzzz' from numbers(1024); + +set max_rows_to_read = 16; + +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc'); +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'def'); +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz'); + +-- invert result +-- this does not work as expected, reading more rows that it should +-- SELECT max(id) FROM bloom_filter WHERE NOT hasToken(s, 'yyy'); + +-- accessing to many rows +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'yyy'); -- { serverError 158 } + +-- this syntax is not supported by tokenbf +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz') == 1; -- { serverError 158 } + +DROP TABLE bloom_filter; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference new file mode 100644 index 00000000000..acb53e80e6d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.reference @@ -0,0 +1 @@ +['9'] diff --git a/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql new file mode 100644 index 00000000000..06de4ec8afb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_extract_all_crash_6627.sql @@ -0,0 +1 @@ +SELECT extractAll('Mozilla/5.0 (Windows NT 10.0; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/75.0.3770.143 YaBrowser/19.7.2.455 Yowser/2.5 Safari/537.36', '[Y][a-zA-Z]{8}/[1-9]([1-9]+)?(((.?)([0-9]+)?){0,4})?'); diff --git a/dbms/tests/queries/0_stateless/00997_set_index_array.reference b/dbms/tests/queries/0_stateless/00997_set_index_array.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_set_index_array.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00997_set_index_array.sql b/dbms/tests/queries/0_stateless/00997_set_index_array.sql new file mode 100644 index 00000000000..c57507ce22d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_set_index_array.sql @@ -0,0 +1,24 @@ +SET allow_experimental_data_skipping_indices = 1; + +DROP TABLE IF EXISTS test.set_array; + +CREATE TABLE test.set_array +( + primary_key String, + index_array Array(UInt64), + INDEX additional_index_array (index_array) TYPE set(10000) GRANULARITY 1 +) ENGINE = MergeTree() +ORDER BY (primary_key); + +INSERT INTO test.set_array +select + toString(intDiv(number, 1000000)) as primary_key, + array(number) as index_array +from system.numbers +limit 10000000; + +SET max_rows_to_read = 8192; + +select count() from test.set_array where has(index_array, 333); + +DROP TABLE test.set_array; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00997_trim.reference b/dbms/tests/queries/0_stateless/00997_trim.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00997_trim.sql b/dbms/tests/queries/0_stateless/00997_trim.sql new file mode 100644 index 00000000000..7519877ec5e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00997_trim.sql @@ -0,0 +1,20 @@ +WITH + '0123456789abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ' AS x, + replaceRegexpAll(x, '.', ' ') AS spaces, + concat(substring(spaces, 1, rand(1) % 62), substring(x, 1, rand(2) % 62), substring(spaces, 1, rand(3) % 62)) AS s, + trimLeft(s) AS sl, + trimRight(s) AS sr, + trimBoth(s) AS t, + replaceRegexpOne(s, '^ +', '') AS slr, + replaceRegexpOne(s, ' +$', '') AS srr, + replaceRegexpOne(s, '^ *(.*?) *$', '\\1') AS tr +SELECT + replaceAll(s, ' ', '_'), + replaceAll(sl, ' ', '_'), + replaceAll(slr, ' ', '_'), + replaceAll(sr, ' ', '_'), + replaceAll(srr, ' ', '_'), + replaceAll(t, ' ', '_'), + replaceAll(tr, ' ', '_') +FROM numbers(100000) +WHERE NOT ((sl = slr) AND (sr = srr) AND (t = tr)) diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index d4ab11be927..b3058a6cdbe 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -46,7 +46,8 @@ export CLICKHOUSE_PORT_INTERSERVER=${CLICKHOUSE_PORT_INTERSERVER:="9009"} export CLICKHOUSE_URL_INTERSERVER=${CLICKHOUSE_URL_INTERSERVER:="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_INTERSERVER}/"} export CLICKHOUSE_CURL_COMMAND=${CLICKHOUSE_CURL_COMMAND:="curl"} -export CLICKHOUSE_CURL=${CLICKHOUSE_CURL:="${CLICKHOUSE_CURL_COMMAND} --max-time 10"} +export CLICKHOUSE_CURL_TIMEOUT=${CLICKHOUSE_CURL_TIMEOUT:="10"} +export CLICKHOUSE_CURL=${CLICKHOUSE_CURL:="${CLICKHOUSE_CURL_COMMAND} --max-time ${CLICKHOUSE_CURL_TIMEOUT}"} export CLICKHOUSE_TMP=${CLICKHOUSE_TMP:="."} mkdir -p ${CLICKHOUSE_TMP} diff --git a/docs/en/operations/settings/query_complexity.md b/docs/en/operations/settings/query_complexity.md index 77699c868b4..c00f2132ebd 100644 --- a/docs/en/operations/settings/query_complexity.md +++ b/docs/en/operations/settings/query_complexity.md @@ -79,7 +79,7 @@ Enables or disables execution of `GROUP BY` clauses in external memory. See [GRO Possible values: -- Maximum volume or RAM (in bytes) that can be used by the single [GROUP BY](../../query_language/select.md#select-group-by-clause) operation. +- Maximum volume of RAM (in bytes) that can be used by the single [GROUP BY](../../query_language/select.md#select-group-by-clause) operation. - 0 — `GROUP BY` in external memory disabled. Default value: 0. diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 93f548bf73c..e366abf2f3d 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -193,18 +193,21 @@ RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... Все таблицы переименовываются под глобальной блокировкой. Переименовывание таблицы является лёгкой операцией. Если вы указали после TO другую базу данных, то таблица будет перенесена в эту базу данных. При этом, директории с базами данных должны быть расположены в одной файловой системе (иначе возвращается ошибка). -## SET +## SET {#query-set} ```sql SET param = value ``` -Позволяет установить настройку `param` в значение `value`. Также можно одним запросом установить все настройки из заданного профиля настроек. Для этого укажите 'profile' в качестве имени настройки. Подробнее смотрите в разделе "Настройки". -Настройка устанавливается на сессию, или на сервер (глобально), если указано `GLOBAL`. -При установке глобальных настроек, эти настройки не применяются к уже запущенной сессии, включая текущую сессию. Она будет использована только для новых сессий. +Устанавливает значение `value` для настройки `param` в текущей сессии. [Конфигурационные параметры сервера](../operations/server_settings/index.md) нельзя изменить подобным образом. -При перезапуске сервера теряются настройки, установленные с помощью `SET`. -Установить настройки, которые переживут перезапуск сервера, можно только с помощью конфигурационного файла сервера. +Можно одним запросом установить все настройки из заданного профиля настроек. + +```sql +SET profile = 'profile-name-from-the-settings-file' +``` + +Подробности смотрите в разделе [Настройки](../operations/settings/settings.md). ## SHOW CREATE TABLE diff --git a/docs/zh/database_engines/index.md b/docs/zh/database_engines/index.md deleted file mode 120000 index bbdb762a4ad..00000000000 --- a/docs/zh/database_engines/index.md +++ /dev/null @@ -1 +0,0 @@ -../../en/database_engines/index.md \ No newline at end of file diff --git a/docs/zh/database_engines/index.md b/docs/zh/database_engines/index.md new file mode 100644 index 00000000000..f8ae05e2520 --- /dev/null +++ b/docs/zh/database_engines/index.md @@ -0,0 +1,11 @@ +# 数据库引擎 + +您使用的所有表都是由数据库引擎所提供的 + +默认情况下,ClickHouse使用自己的数据库引擎,该引擎提供可配置的[表引擎](../operations/table_engines/index.md)和[所有支持的SQL语法](../query_language/syntax.md). + +除此之外,您还可以选择使用以下的数据库引擎: + +- [MySQL](mysql.md) + +[来源文章](https://clickhouse.yandex/docs/en/database_engines/) diff --git a/docs/zh/database_engines/mysql.md b/docs/zh/database_engines/mysql.md deleted file mode 120000 index 51ac4126e2d..00000000000 --- a/docs/zh/database_engines/mysql.md +++ /dev/null @@ -1 +0,0 @@ -../../en/database_engines/mysql.md \ No newline at end of file diff --git a/docs/zh/database_engines/mysql.md b/docs/zh/database_engines/mysql.md new file mode 100644 index 00000000000..38dfcb5ef64 --- /dev/null +++ b/docs/zh/database_engines/mysql.md @@ -0,0 +1,124 @@ +# MySQL + +MySQL引擎用于将远程的MySQL服务器中的表映射到ClickHouse中,并允许您对表进行`INSERT`和`SELECT`查询,以方便您在ClickHouse与MySQL之间进行数据交换。 + +`MySQL`数据库引擎会将对其的查询转换为MySQL语法并发送到MySQL服务器中,因此您可以执行诸如`SHOW TABLES`或`SHOW CREATE TABLE`之类的操作。 + +但您无法对其执行以下操作: + +- `ATTACH`/`DETACH` +- `DROP` +- `RENAME` +- `CREATE TABLE` +- `ALTER` + + +## CREATE DATABASE + +``` sql +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +ENGINE = MySQL('host:port', 'database', 'user', 'password') +``` + +**MySQL数据库引擎参数** + +- `host:port` — 链接的MySQL地址。 +- `database` — 链接的MySQL数据库。 +- `user` — 链接的MySQL用户。 +- `password` — 链接的MySQL用户密码。 + + +## 支持的类型对应 + +MySQL | ClickHouse +------|------------ +UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) +TINYINT | [Int8](../data_types/int_uint.md) +UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) +SMALLINT | [Int16](../data_types/int_uint.md) +UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) +INT, MEDIUMINT | [Int32](../data_types/int_uint.md) +UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) +BIGINT | [Int64](../data_types/int_uint.md) +FLOAT | [Float32](../data_types/float.md) +DOUBLE | [Float64](../data_types/float.md) +DATE | [Date](../data_types/date.md) +DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) +BINARY | [FixedString](../data_types/fixedstring.md) + +其他的MySQL数据类型将全部都转换为[String](../data_types/string.md)。 + +同时以上的所有类型都支持[Nullable](../data_types/nullable.md)。 + + +## 使用示例 + +在MySQL中创建表: + +``` +mysql> USE test; +Database changed + +mysql> CREATE TABLE `mysql_table` ( + -> `int_id` INT NOT NULL AUTO_INCREMENT, + -> `float` FLOAT NOT NULL, + -> PRIMARY KEY (`int_id`)); +Query OK, 0 rows affected (0,09 sec) + +mysql> insert into mysql_table (`int_id`, `float`) VALUES (1,2); +Query OK, 1 row affected (0,00 sec) + +mysql> select * from mysql_table; ++--------+-------+ +| int_id | value | ++--------+-------+ +| 1 | 2 | ++--------+-------+ +1 row in set (0,00 sec) +``` + +在ClickHouse中创建MySQL类型的数据库,同时与MySQL服务器交换数据: + +```sql +CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') +``` +```sql +SHOW DATABASES +``` +```text +┌─name─────┐ +│ default │ +│ mysql_db │ +│ system │ +└──────────┘ +``` +```sql +SHOW TABLES FROM mysql_db +``` +```text +┌─name─────────┐ +│ mysql_table │ +└──────────────┘ +``` +```sql +SELECT * FROM mysql_db.mysql_table +``` +```text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +└────────┴───────┘ +``` +```sql +INSERT INTO mysql_db.mysql_table VALUES (3,4) +``` +```sql +SELECT * FROM mysql_db.mysql_table +``` +```text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +[来源文章](https://clickhouse.yandex/docs/en/database_engines/mysql/) diff --git a/libs/libcommon/include/common/find_symbols.h b/libs/libcommon/include/common/find_symbols.h index 68b49397683..162c73251fa 100644 --- a/libs/libcommon/include/common/find_symbols.h +++ b/libs/libcommon/include/common/find_symbols.h @@ -17,7 +17,7 @@ * but with the following differencies: * - works with any memory ranges, including containing zero bytes; * - doesn't require terminating zero byte: end of memory range is passed explicitly; - * - if not found, returns pointer to end instead of NULL; + * - if not found, returns pointer to end instead of nullptr; * - maximum number of symbols to search is 16. * * Uses SSE 2 in case of small number of symbols for search and SSE 4.2 in the case of large number of symbols, @@ -65,115 +65,154 @@ inline __m128i mm_is_in(__m128i bytes) } #endif - -template -inline const char * find_first_symbols_sse2(const char * begin, const char * end) +template +bool maybe_negate(bool x) { + if constexpr (positive) + return x; + else + return !x; +} + +template +uint16_t maybe_negate(uint16_t x) +{ + if constexpr (positive) + return x; + else + return ~x; +} + +enum class ReturnMode +{ + End, + Nullptr, +}; + + +template +inline const char * find_first_symbols_sse2(const char * const begin, const char * const end) +{ + const char * pos = begin; + #if defined(__SSE2__) - for (; begin + 15 < end; begin += 16) + for (; pos + 15 < end; pos += 16) { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(begin)); + __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos)); __m128i eq = mm_is_in(bytes); - uint16_t bit_mask = _mm_movemask_epi8(eq); + uint16_t bit_mask = maybe_negate(uint16_t(_mm_movemask_epi8(eq))); if (bit_mask) - return begin + __builtin_ctz(bit_mask); + return pos + __builtin_ctz(bit_mask); } #endif - for (; begin < end; ++begin) - if (is_in(*begin)) - return begin; - return end; + for (; pos < end; ++pos) + if (maybe_negate(is_in(*pos))) + return pos; + + return return_mode == ReturnMode::End ? end : nullptr; } -template -inline const char * find_last_symbols_or_null_sse2(const char * begin, const char * end) +template +inline const char * find_last_symbols_sse2(const char * const begin, const char * const end) { + const char * pos = end; + #if defined(__SSE2__) - for (; end - 16 >= begin; end -= 16) /// Assuming the pointer cannot overflow. Assuming we can compare these pointers. + for (; pos - 16 >= begin; pos -= 16) /// Assuming the pointer cannot overflow. Assuming we can compare these pointers. { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(end - 16)); + __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos - 16)); __m128i eq = mm_is_in(bytes); - uint16_t bit_mask = _mm_movemask_epi8(eq); + uint16_t bit_mask = maybe_negate(uint16_t(_mm_movemask_epi8(eq))); if (bit_mask) - return end - 1 - (__builtin_clz(bit_mask) - 16); /// because __builtin_clz works with mask as uint32. + return pos - 1 - (__builtin_clz(bit_mask) - 16); /// because __builtin_clz works with mask as uint32. } #endif - --end; - for (; end >= begin; --end) - if (is_in(*end)) - return end; + --pos; + for (; pos >= begin; --pos) + if (maybe_negate(is_in(*pos))) + return pos; - return nullptr; + return return_mode == ReturnMode::End ? end : nullptr; } -template -inline const char * find_first_symbols_sse42_impl(const char * begin, const char * end) +inline const char * find_first_symbols_sse42_impl(const char * const begin, const char * const end) { + const char * pos = begin; + #if defined(__SSE4_2__) #define MODE (_SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_ANY | _SIDD_LEAST_SIGNIFICANT) __m128i set = _mm_setr_epi8(c01, c02, c03, c04, c05, c06, c07, c08, c09, c10, c11, c12, c13, c14, c15, c16); - for (; begin + 15 < end; begin += 16) + for (; pos + 15 < end; pos += 16) { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(begin)); + __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos)); - if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE)) - return begin + _mm_cmpestri(set, num_chars, bytes, 16, MODE); + if constexpr (positive) + { + if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE)) + return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE); + } + else + { + if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY)) + return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY); + } } #undef MODE #endif - for (; begin < end; ++begin) - if ( (num_chars >= 1 && *begin == c01) - || (num_chars >= 2 && *begin == c02) - || (num_chars >= 3 && *begin == c03) - || (num_chars >= 4 && *begin == c04) - || (num_chars >= 5 && *begin == c05) - || (num_chars >= 6 && *begin == c06) - || (num_chars >= 7 && *begin == c07) - || (num_chars >= 8 && *begin == c08) - || (num_chars >= 9 && *begin == c09) - || (num_chars >= 10 && *begin == c10) - || (num_chars >= 11 && *begin == c11) - || (num_chars >= 12 && *begin == c12) - || (num_chars >= 13 && *begin == c13) - || (num_chars >= 14 && *begin == c14) - || (num_chars >= 15 && *begin == c15) - || (num_chars >= 16 && *begin == c16)) - return begin; - return end; + for (; pos < end; ++pos) + if ( (num_chars >= 1 && maybe_negate(*pos == c01)) + || (num_chars >= 2 && maybe_negate(*pos == c02)) + || (num_chars >= 3 && maybe_negate(*pos == c03)) + || (num_chars >= 4 && maybe_negate(*pos == c04)) + || (num_chars >= 5 && maybe_negate(*pos == c05)) + || (num_chars >= 6 && maybe_negate(*pos == c06)) + || (num_chars >= 7 && maybe_negate(*pos == c07)) + || (num_chars >= 8 && maybe_negate(*pos == c08)) + || (num_chars >= 9 && maybe_negate(*pos == c09)) + || (num_chars >= 10 && maybe_negate(*pos == c10)) + || (num_chars >= 11 && maybe_negate(*pos == c11)) + || (num_chars >= 12 && maybe_negate(*pos == c12)) + || (num_chars >= 13 && maybe_negate(*pos == c13)) + || (num_chars >= 14 && maybe_negate(*pos == c14)) + || (num_chars >= 15 && maybe_negate(*pos == c15)) + || (num_chars >= 16 && maybe_negate(*pos == c16))) + return pos; + return return_mode == ReturnMode::End ? end : nullptr; } -template +template inline const char * find_first_symbols_sse42(const char * begin, const char * end) { - return find_first_symbols_sse42_impl(begin, end); + return find_first_symbols_sse42_impl(begin, end); } /// NOTE No SSE 4.2 implementation for find_last_symbols_or_null. Not worth to do. -template +template inline const char * find_first_symbols_dispatch(const char * begin, const char * end) { #if defined(__SSE4_2__) if (sizeof...(symbols) >= 5) - return find_first_symbols_sse42(begin, end); + return find_first_symbols_sse42(begin, end); else #endif - return find_first_symbols_sse2(begin, end); + return find_first_symbols_sse2(begin, end); } } @@ -182,7 +221,7 @@ inline const char * find_first_symbols_dispatch(const char * begin, const char * template inline const char * find_first_symbols(const char * begin, const char * end) { - return detail::find_first_symbols_dispatch(begin, end); + return detail::find_first_symbols_dispatch(begin, end); } /// Returning non const result for non const arguments. @@ -190,18 +229,66 @@ inline const char * find_first_symbols(const char * begin, const char * end) template inline char * find_first_symbols(char * begin, char * end) { - return const_cast(detail::find_first_symbols_dispatch(begin, end)); + return const_cast(detail::find_first_symbols_dispatch(begin, end)); +} + +template +inline const char * find_first_not_symbols(const char * begin, const char * end) +{ + return detail::find_first_symbols_dispatch(begin, end); +} + +template +inline char * find_first_not_symbols(char * begin, char * end) +{ + return const_cast(detail::find_first_symbols_dispatch(begin, end)); +} + +template +inline const char * find_first_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_first_symbols_dispatch(begin, end); +} + +template +inline char * find_first_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_first_symbols_dispatch(begin, end)); +} + +template +inline const char * find_first_not_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_first_symbols_dispatch(begin, end); +} + +template +inline char * find_first_not_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_first_symbols_dispatch(begin, end)); } template inline const char * find_last_symbols_or_null(const char * begin, const char * end) { - return detail::find_last_symbols_or_null_sse2(begin, end); + return detail::find_last_symbols_sse2(begin, end); } template inline char * find_last_symbols_or_null(char * begin, char * end) { - return const_cast(detail::find_last_symbols_or_null_sse2(begin, end)); + return const_cast(detail::find_last_symbols_sse2(begin, end)); +} + +template +inline const char * find_last_not_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_last_symbols_sse2(begin, end); +} + +template +inline char * find_last_not_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_last_symbols_sse2(begin, end)); } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index fed4b6b8670..deed481f043 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -25,3 +25,9 @@ find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | grep -vP 'Compiler|build' | xargs grep $@ -P '}\s*//+\s*namespace\s*' + +# Broken symlinks +find -L $ROOT_PATH -type l | grep -v contrib && echo "^ Broken symlinks found" + +# Double whitespaces +find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | while read i; do $ROOT_PATH/utils/check-style/double-whitespaces.pl < $i || echo -e "^ File $i contains double whitespaces\n"; done diff --git a/utils/check-style/double-whitespaces.pl b/utils/check-style/double-whitespaces.pl new file mode 100755 index 00000000000..47b03cb74ab --- /dev/null +++ b/utils/check-style/double-whitespaces.pl @@ -0,0 +1,33 @@ +#!/usr/bin/perl + +use strict; + +# Find double whitespace such as "a, b, c" that looks very ugly and annoying. +# But skip double whitespaces if they are used as an alignment - by comparing to surrounding lines. + +my @array; + +while (<>) +{ + push @array, $_; +} + +my $ret = 0; + +for (my $i = 1; $i < $#array; ++$i) +{ + if ($array[$i] =~ ',( {2,3})[^ /]') + { + # https://stackoverflow.com/questions/87380/how-can-i-find-the-location-of-a-regex-match-in-perl + + if ((substr($array[$i - 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) # whitespaces are not part of alignment + && (substr($array[$i + 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) + && $array[$i] !~ /(-?\d+\w*,\s+){3,}/) # this is not a number table like { 10, -1, 2 } + { + print(($i + 1) . ":" . $array[$i]); + $ret = 1; + } + } +} + +exit $ret;