From 3c1e05eb53c27b8659725ab0b3d777e8895fdb6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 00:20:47 +0200 Subject: [PATCH] Remove Arcadia --- src/AggregateFunctions/UniquesHashSet.h | 2 +- src/Common/Config/ConfigProcessor.h | 3 +- src/Common/HashTable/Hash.h | 2 +- src/Common/OptimizedRegularExpression.cpp | 2 +- src/Common/StringUtils/CMakeLists.txt | 2 +- src/Common/ZooKeeper/ZooKeeperCommon.h | 2 +- src/Common/ZooKeeper/examples/CMakeLists.txt | 3 - .../examples/zk_many_watches_reconnect.cpp | 66 --------- src/Common/mysqlxx/mysqlxx/mysqlxx.h | 6 +- src/Common/parseAddress.h | 4 +- src/Functions/DateTimeTransforms.h | 2 +- src/Functions/FunctionFactory.h | 2 +- src/Functions/FunctionMathUnary.h | 2 +- src/Functions/FunctionsEmbeddedDictionaries.h | 2 +- src/Functions/FunctionsExternalDictionaries.h | 2 +- src/Functions/FunctionsVisitParam.h | 2 +- src/Functions/URL/FunctionsURL.h | 2 +- src/Functions/array/arrayEnumerateRanked.h | 2 +- src/Functions/array/arrayIndex.h | 4 +- src/Functions/extractTextFromHTML.cpp | 2 +- src/Functions/timeSlots.cpp | 2 +- src/IO/ReadHelpers.cpp | 2 +- src/IO/tests/gtest_s3_uri.cpp | 34 ++--- ...OptimizeIfWithConstantConditionVisitor.cpp | 2 +- src/Parsers/ASTFunction.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- .../Impl/JSONAsStringRowInputFormat.cpp | 1 + .../Algorithms/tests/gtest_graphite.cpp | 8 +- .../Transforms/ExpressionTransform.h | 2 +- src/Storages/examples/CMakeLists.txt | 13 -- .../examples/remove_symlink_directory.cpp | 35 ----- .../examples/transform_part_zk_nodes.cpp | 131 ------------------ 32 files changed, 49 insertions(+), 299 deletions(-) delete mode 100644 src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp delete mode 100644 src/Storages/examples/remove_symlink_directory.cpp delete mode 100644 src/Storages/examples/transform_part_zk_nodes.cpp diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index 6837803c67d..8648f6e2500 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -59,7 +59,7 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, - * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. + * stored in many places on disks (in many companies), so it continues to be used. */ struct UniquesHashSetDefaultHash { diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 6c642690945..0e1d0facf48 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -55,7 +55,7 @@ public: /// 2) Determine the includes file from the config: /path2/metrika.xml /// If this path is not configured, use /etc/metrika.xml /// 3) Replace elements matching the "" pattern with - /// "contents of the yandex/bar element in metrika.xml" + /// "contents of the clickhouse/bar element in metrika.xml" /// 4) If zk_node_cache is non-NULL, replace elements matching the "" pattern with /// "contents of the /bar ZooKeeper node". /// If has_zk_includes is non-NULL and there are such elements, set has_zk_includes to true. @@ -137,4 +137,3 @@ private: }; } - diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 5dbeeecf96b..3cf8978f418 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -13,7 +13,7 @@ * * Example: when we do aggregation by the visitor ID, the performance increase is more than 5 times. * This is because of following reasons: - * - in Yandex, visitor identifier is an integer that has timestamp with seconds resolution in lower bits; + * - in Metrica web analytics system, visitor identifier is an integer that has timestamp with seconds resolution in lower bits; * - in typical implementation of standard library, hash function for integers is trivial and just use lower bits; * - traffic is non-uniformly distributed across a day; * - we are using open-addressing linear probing hash tables that are most critical to hash function quality, diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 1464923e6ab..da348adbe31 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -28,7 +28,7 @@ void OptimizedRegularExpressionImpl::analyze( * in which all metacharacters are escaped, * and also if there are no '|' outside the brackets, * and also avoid substrings of the form `http://` or `www` and some other - * (this is the hack for typical use case in Yandex.Metrica). + * (this is the hack for typical use case in web analytics applications). */ const char * begin = regexp.data(); const char * pos = begin; diff --git a/src/Common/StringUtils/CMakeLists.txt b/src/Common/StringUtils/CMakeLists.txt index 4eedbf4842b..57c196d335c 100644 --- a/src/Common/StringUtils/CMakeLists.txt +++ b/src/Common/StringUtils/CMakeLists.txt @@ -1,4 +1,4 @@ -# These files are located in separate library, because they are used by Yandex.Metrika code +# These files are located in separate library, because they are used by separate products # in places when no dependency on whole "dbms" library is possible. include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index e51bea3f7f8..532488c08f8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -45,7 +45,7 @@ struct ZooKeeperResponse : virtual Response using ZooKeeperResponsePtr = std::shared_ptr; -/// Exposed in header file for Yandex.Metrica code. +/// Exposed in header file for some external code. struct ZooKeeperRequest : virtual Request { XID xid = 0; diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index 8bec951e24f..b449b172605 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -7,8 +7,5 @@ target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zoo add_executable(zkutil_test_async zkutil_test_async.cpp) target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) -add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp) -target_link_libraries (zk_many_watches_reconnect PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_config) - add_executable (zookeeper_impl zookeeper_impl.cpp) target_link_libraries (zookeeper_impl PRIVATE clickhouse_common_zookeeper_no_log) diff --git a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp deleted file mode 100644 index cf819121234..00000000000 --- a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp +++ /dev/null @@ -1,66 +0,0 @@ -#include -#include -#include -#include - -/// A tool for reproducing https://issues.apache.org/jira/browse/ZOOKEEPER-706 -/// Original libzookeeper can't reconnect the session if the length of SET_WATCHES message -/// exceeds jute.maxbuffer (0xfffff by default). -/// This happens when the number of watches exceeds ~29000. -/// -/// Session reconnect can be caused by forbidding packets to the current zookeeper server, e.g. -/// sudo ip6tables -A OUTPUT -d mtzoo01it.haze.yandex.net -j REJECT - -const size_t N_THREADS = 100; - -int main(int argc, char ** argv) -{ - try - { - if (argc != 3) - { - std::cerr << "usage: " << argv[0] << " " << std::endl; - return 3; - } - - DB::ConfigProcessor processor(argv[1], false, true); - auto config = processor.loadConfig().configuration; - zkutil::ZooKeeper zk(*config, "zookeeper", nullptr); - zkutil::EventPtr watch = std::make_shared(); - - /// NOTE: setting watches in multiple threads because doing it in a single thread is too slow. - size_t watches_per_thread = std::stoull(argv[2]) / N_THREADS; - std::vector threads; - for (size_t i_thread = 0; i_thread < N_THREADS; ++i_thread) - { - threads.emplace_back([&, i_thread] - { - for (size_t i = 0; i < watches_per_thread; ++i) - zk.exists("/clickhouse/nonexistent_node" + std::to_string(i * N_THREADS + i_thread), nullptr, watch); - }); - } - for (size_t i_thread = 0; i_thread < N_THREADS; ++i_thread) - threads[i_thread].join(); - - while (true) - { - std::cerr << "WAITING..." << std::endl; - sleep(10); - } - } - catch (Poco::Exception & e) - { - std::cerr << "Exception: " << e.displayText() << std::endl; - return 1; - } - catch (std::exception & e) - { - std::cerr << "std::exception: " << e.what() << std::endl; - return 3; - } - catch (...) - { - std::cerr << "Some exception" << std::endl; - return 2; - } -} diff --git a/src/Common/mysqlxx/mysqlxx/mysqlxx.h b/src/Common/mysqlxx/mysqlxx/mysqlxx.h index 785d4361fd7..0b6cf5e7c1a 100644 --- a/src/Common/mysqlxx/mysqlxx/mysqlxx.h +++ b/src/Common/mysqlxx/mysqlxx/mysqlxx.h @@ -10,10 +10,10 @@ /** 'mysqlxx' - very simple library for replacement of 'mysql++' library. * - * For whatever reason, in Yandex.Metrica, back in 2008, 'mysql++' library was used. + * For whatever reason, in Metrica web analytics system, back in 2008, 'mysql++' library was used. * There are the following shortcomings of 'mysql++': * 1. Too rich functionality: most of it is not used. - * 2. Low performance (when used for Yandex.Metrica). + * 2. Low performance (when used for Metrica). * * Low performance is caused by the following reasons: * @@ -50,7 +50,7 @@ * And for the sake of simplicity, some functions work only with certain assumptions, * or with slightly different semantic than in mysql++. * And we don't care about cross-platform usage of mysqlxx. - * These assumptions are specific for Yandex.Metrica. Your mileage may vary. + * These assumptions are specific for Metrica. Your mileage may vary. * * mysqlxx could not be considered as separate full-featured library, * because it is developed from the principle - "everything that we don't need is not implemented". diff --git a/src/Common/parseAddress.h b/src/Common/parseAddress.h index 86340982c67..297934a6379 100644 --- a/src/Common/parseAddress.h +++ b/src/Common/parseAddress.h @@ -13,8 +13,8 @@ namespace DB * Otherwise, an exception is thrown. * * Examples: - * yandex.ru - returns "yandex.ru" and default_port - * yandex.ru:80 - returns "yandex.ru" and 80 + * clickhouse.com - returns "clickhouse.com" and default_port + * clickhouse.com:80 - returns "clickhouse.com" and 80 * [2a02:6b8:a::a]:80 - returns [2a02:6b8:a::a] and 80; note that square brackets remain in returned host. */ std::pair parseAddress(const std::string & str, UInt16 default_port); diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 3a7baf45adc..5d1bcaf48cf 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -539,7 +539,7 @@ struct ToStartOfFifteenMinutesImpl using FactorTransform = ZeroTransform; }; -/// Round to start of half-an-hour length interval with unspecified offset. This transform is specific for Yandex.Metrica. +/// Round to start of half-an-hour length interval with unspecified offset. This transform is specific for Metrica web analytics system. struct TimeSlotImpl { static constexpr auto name = "timeSlot"; diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index 898a463cd58..13b14559ec4 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -40,7 +40,7 @@ public: registerFunction(name, &Function::create, case_sensitiveness); } - /// This function is used by YQL - internal Yandex product that depends on ClickHouse by source code. + /// This function is used by YQL - innovative transactional DBMS that depends on ClickHouse by source code. std::vector getAllNames() const; bool has(const std::string & name) const; diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index fa10c004e87..bd656db792b 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -10,7 +10,7 @@ #include "config_functions.h" -/** FastOps is a fast vector math library from Mikhail Parakhin (former Yandex CTO), +/** FastOps is a fast vector math library from Mikhail Parakhin, https://www.linkedin.com/in/mikhail-parakhin/ * Enabled by default. */ #if USE_FASTOPS diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index c6ea886b4a8..20be3ee3cce 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -33,7 +33,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/** Functions using Yandex.Metrica dictionaries +/** Functions using deprecated dictionaries * - dictionaries of regions, operating systems, search engines. * * Climb up the tree to a certain level. diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 6a701d7b864..189ec7321c1 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -195,7 +195,7 @@ public: * that is the initiator of a distributed query, * in the case when the function will be invoked for real data only at the remote servers. * This feature is controversial and implemented specially - * for backward compatibility with the case in Yandex Banner System. + * for backward compatibility with the case in the Banner System application. */ if (input_rows_count == 0) return result_type->createColumn(); diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index 362c3bcd693..09fcf8659ed 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -12,7 +12,7 @@ /** Functions for retrieving "visit parameters". - * Visit parameters in Yandex.Metrika are a special kind of JSONs. + * Visit parameters in Metrica web analytics system are a special kind of JSONs. * These functions are applicable to almost any JSONs. * Implemented via templates from FunctionsStringSearch.h. * diff --git a/src/Functions/URL/FunctionsURL.h b/src/Functions/URL/FunctionsURL.h index 297b62ca256..a0f106742fb 100644 --- a/src/Functions/URL/FunctionsURL.h +++ b/src/Functions/URL/FunctionsURL.h @@ -45,7 +45,7 @@ namespace DB * Remove specified parameter from URL. * cutURLParameter(URL, name) * - * Get array of URL 'hierarchy' as in Yandex.Metrica tree-like reports. See docs. + * Get array of URL 'hierarchy' as in web-analytics tree-like reports. See the docs. * URLHierarchy(URL) */ diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index d6a62a966ae..ac3936af0fd 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -15,7 +15,7 @@ // for better debug: #include /** The function will enumerate distinct values of the passed multidimensional arrays looking inside at the specified depths. - * This is very unusual function made as a special order for Yandex.Metrica. + * This is very unusual function made as a special order for our dear customer - Metrica web analytics system. * * arrayEnumerateUniqRanked(['hello', 'world', 'hello']) = [1, 1, 2] * - it returns similar structured array containing number of occurrence of the corresponding value. diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index 8b42b99cd69..0dbbe5e41b6 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -718,9 +718,7 @@ private: /** * Catches arguments of type LowCardinality(T) (left) and U (right). * - * The perftests - * https://clickhouse-test-reports.s3.yandex.net/12550/2d27fa0fa8c198a82bf1fe3625050ccf56695976/integration_tests_(release).html - * showed that the amount of action needed to convert the non-constant right argument to the index column + * The perftests showed that the amount of action needed to convert the non-constant right argument to the index column * (similar to the left one's) is significantly higher than converting the array itself to an ordinary column. * * So, in terms of performance it's more optimal to fall back to default implementation and catch only constant diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index b38ea74d6ce..f321a59f734 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -53,7 +53,7 @@ * * Usage example: * - * SELECT extractTextFromHTML(html) FROM url('https://yandex.ru/', RawBLOB, 'html String') + * SELECT extractTextFromHTML(html) FROM url('https://github.com/ClickHouse/ClickHouse', RawBLOB, 'html String') * * - ClickHouse has embedded web browser. */ diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index 34af410befa..a19ccf62565 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -29,7 +29,7 @@ namespace * For example, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]. * This is necessary to search for hits that are part of the corresponding visit. * - * This is obsolete function. It was developed for Yandex.Metrica, but no longer used in Yandex. + * This is obsolete function. It was developed for Metrica web analytics system, but the art of its usage has been forgotten. * But this function was adopted by wider audience. */ diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index bf3cccccab8..8aee5713cdc 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -343,7 +343,7 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf) && decoded_char != '"' && decoded_char != '`' /// MySQL style identifiers && decoded_char != '/' /// JavaScript in HTML - && decoded_char != '=' /// Yandex's TSKV + && decoded_char != '=' /// TSKV format invented somewhere && !isControlASCII(decoded_char)) { s.push_back('\\'); diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 7ee72069e57..20d19437c64 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -16,36 +16,36 @@ class S3UriTest : public testing::TestWithParam TEST(S3UriTest, validPatterns) { { - S3::URI uri(Poco::URI("https://jokserfn.s3.yandexcloud.net/")); - ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://jokserfn.s3.amazonaws.com/")); + ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("", uri.key); ASSERT_EQ(true, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://s3.yandexcloud.net/jokserfn/")); - ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://s3.amazonaws.com/jokserfn/")); + ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("", uri.key); ASSERT_EQ(false, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://yandexcloud.net/bucket/")); - ASSERT_EQ("https://yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://amazonaws.com/bucket/")); + ASSERT_EQ("https://amazonaws.com", uri.endpoint); ASSERT_EQ("bucket", uri.bucket); ASSERT_EQ("", uri.key); ASSERT_EQ(false, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://jokserfn.s3.yandexcloud.net/data")); - ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://jokserfn.s3.amazonaws.com/data")); + ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("data", uri.key); ASSERT_EQ(true, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://storage.yandexcloud.net/jokserfn/data")); - ASSERT_EQ("https://storage.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://storage.amazonaws.com/jokserfn/data")); + ASSERT_EQ("https://storage.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("data", uri.key); ASSERT_EQ(false, uri.is_virtual_hosted_style); @@ -97,13 +97,13 @@ INSTANTIATE_TEST_SUITE_P( S3UriTest, testing::Values( "https:///", - "https://.s3.yandexcloud.net/key", - "https://s3.yandexcloud.net/key", - "https://jokserfn.s3yandexcloud.net/key", - "https://s3.yandexcloud.net//", - "https://yandexcloud.net/", - "https://yandexcloud.net//", - "https://yandexcloud.net//key")); + "https://.s3.amazonaws.com/key", + "https://s3.amazonaws.com/key", + "https://jokserfn.s3amazonaws.com/key", + "https://s3.amazonaws.com//", + "https://amazonaws.com/", + "https://amazonaws.com//", + "https://amazonaws.com//key")); } diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 802bf4e43ce..28b19f6670d 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -28,7 +28,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v } /// cast of numeric constant in condition to UInt8 - /// Note: this solution is ad-hoc and only implemented for yandex.metrica use case. + /// Note: this solution is ad-hoc and only implemented for metrica use case (one of the best customers). /// We should allow any constant condition (or maybe remove this optimization completely) later. if (const auto * function = condition->as()) { diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index f9a5c7be75f..b86929b054c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -166,7 +166,7 @@ ASTPtr ASTFunction::toLiteral() const * Motivation: most people are unaware that _ is a metacharacter and forgot to properly escape it with two backslashes. * With highlighting we make it clearly obvious. * - * Another case is regexp match. Suppose the user types match(URL, 'www.yandex.ru'). It often means that the user is unaware that . is a metacharacter. + * Another case is regexp match. Suppose the user types match(URL, 'www.clickhouse.com'). It often means that the user is unaware that . is a metacharacter. */ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const IAST::FormatSettings & settings, const char * metacharacters) { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index cd07e304a39..29c7846283e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1727,7 +1727,7 @@ const char * ParserAlias::restricted_keywords[] = "NOT", "OFFSET", "ON", - "ONLY", /// YQL synonym for ANTI. Note: YQL is the name of one of Yandex proprietary languages, completely unrelated to ClickHouse. + "ONLY", /// YQL's synonym for ANTI. Note: YQL is the name of one of proprietary languages, completely unrelated to ClickHouse. "ORDER", "PREWHERE", "RIGHT", diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 9bf1682b77e..e31006ff0f6 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -52,6 +52,7 @@ void JSONAsRowInputFormat::readSuffix() { assertChar(']', *buf); skipWhitespaceIfAny(*buf); + data_in_square_brackets = false; } if (!buf->eof() && *buf->position() == ';') { diff --git a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp index 1d739bf566a..9160ad6e0fa 100644 --- a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp +++ b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp @@ -149,7 +149,7 @@ TEST(GraphiteTest, testSelectPattern) using namespace std::literals; std::string - xml(R"END( + xml(R"END( \.sum$ @@ -210,7 +210,7 @@ TEST(GraphiteTest, testSelectPattern) - + )END"); // Retentions must be ordered by 'age' descending. @@ -370,7 +370,7 @@ TEST(GraphiteTest, testSelectPatternTyped) using namespace std::literals; std::string - xml(R"END( + xml(R"END( plain @@ -488,7 +488,7 @@ TEST(GraphiteTest, testSelectPatternTyped) - + )END"); // Retentions must be ordered by 'age' descending. diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index ea73c8fb1da..791c7d7ba73 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -12,7 +12,7 @@ class ActionsDAG; /** Executes a certain expression over the block. * The expression consists of column identifiers from the block, constants, common functions. - * For example: hits * 2 + 3, url LIKE '%yandex%' + * For example: hits * 2 + 3, url LIKE '%clickhouse%' * The expression processes each row independently of the others. */ class ExpressionTransform final : public ISimpleTransform diff --git a/src/Storages/examples/CMakeLists.txt b/src/Storages/examples/CMakeLists.txt index 103972a106f..ca85ca9d98f 100644 --- a/src/Storages/examples/CMakeLists.txt +++ b/src/Storages/examples/CMakeLists.txt @@ -1,6 +1,3 @@ -add_executable (remove_symlink_directory remove_symlink_directory.cpp) -target_link_libraries (remove_symlink_directory PRIVATE dbms) - add_executable (merge_selector merge_selector.cpp) target_link_libraries (merge_selector PRIVATE dbms) @@ -12,13 +9,3 @@ target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse add_executable (get_abandonable_lock_in_all_partitions get_abandonable_lock_in_all_partitions.cpp) target_link_libraries (get_abandonable_lock_in_all_partitions PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) - -add_executable (transform_part_zk_nodes transform_part_zk_nodes.cpp) -target_link_libraries (transform_part_zk_nodes - PRIVATE - boost::program_options - clickhouse_common_config - clickhouse_common_zookeeper - dbms - string_utils -) diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp deleted file mode 100644 index db436c0a608..00000000000 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ /dev/null @@ -1,35 +0,0 @@ -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; - -namespace DB -{ - namespace ErrorCodes - { - extern const int SYSTEM_ERROR; - } -} - -int main(int, char **) -try -{ - fs::path dir("./test_dir/"); - fs::create_directories(dir); - FS::createFile("./test_dir/file"); - - if (0 != symlink("./test_dir", "./test_link")) - DB::throwFromErrnoWithPath("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR); - - fs::rename("./test_link", "./test_link2"); - fs::remove_all("./test_link2"); - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(false) << "\n"; - return 1; -} diff --git a/src/Storages/examples/transform_part_zk_nodes.cpp b/src/Storages/examples/transform_part_zk_nodes.cpp deleted file mode 100644 index 3cbcc76190e..00000000000 --- a/src/Storages/examples/transform_part_zk_nodes.cpp +++ /dev/null @@ -1,131 +0,0 @@ -#include -#include -#include -#include -#include - -#include - -#include -#include - - -int main(int argc, char ** argv) -try -{ - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "produce help message") - ("address,a", boost::program_options::value()->required(), - "addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181") - ("path,p", boost::program_options::value()->required(), - "where to start") - ; - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Transform contents of part nodes in ZooKeeper to more compact storage scheme." << std::endl; - std::cout << "Usage: " << argv[0] << " [options]" << std::endl; - std::cout << desc << std::endl; - return 1; - } - - zkutil::ZooKeeper zookeeper(options.at("address").as()); - - std::string initial_path = options.at("path").as(); - - struct Node - { - Node( - std::string path_, - std::future get_future_, - std::future children_future_, - Node * parent_) - : path(std::move(path_)) - , get_future(std::move(get_future_)) - , children_future(std::move(children_future_)) - , parent(parent_) - { - } - - std::string path; - std::future get_future; - std::future children_future; - - Node * parent = nullptr; - std::future set_future; - }; - - std::list nodes_queue; - nodes_queue.emplace_back( - initial_path, zookeeper.asyncGet(initial_path), zookeeper.asyncGetChildren(initial_path), nullptr); - - for (auto it = nodes_queue.begin(); it != nodes_queue.end(); ++it) - { - Coordination::GetResponse get_response; - Coordination::ListResponse children_response; - try - { - get_response = it->get_future.get(); - children_response = it->children_future.get(); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::Error::ZNONODE) - continue; - throw; - } - - if (get_response.stat.ephemeralOwner) - continue; - - if (it->path.find("/parts/") != std::string::npos - && !endsWith(it->path, "/columns") - && !endsWith(it->path, "/checksums")) - { - /// The node is related to part. - - /// If it is the part in old format (the node contains children) - convert it to the new format. - if (!children_response.names.empty()) - { - auto part_header = DB::ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes( - zookeeper.get(it->path + "/columns"), zookeeper.get(it->path + "/checksums")); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeRemoveRequest(it->path + "/columns", -1)); - ops.emplace_back(zkutil::makeRemoveRequest(it->path + "/checksums", -1)); - ops.emplace_back(zkutil::makeSetRequest(it->path, part_header.toString(), -1)); - - it->set_future = zookeeper.asyncMulti(ops); - } - } - else - { - /// Recursively add children to the queue. - for (const auto & name : children_response.names) - { - std::string child_path = it->path == "/" ? it->path + name : it->path + '/' + name; - nodes_queue.emplace_back( - child_path, zookeeper.asyncGet(child_path), zookeeper.asyncGetChildren(child_path), - &(*it)); - } - } - } - - for (auto & node : nodes_queue) - { - if (node.set_future.valid()) - { - node.set_future.get(); - std::cerr << node.path << " changed!" << std::endl; - } - } -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; - throw; -}