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;
-}