Remove Arcadia

This commit is contained in:
Alexey Milovidov 2022-04-16 00:20:47 +02:00 committed by alesapin
parent 0118d98e93
commit 3c1e05eb53
32 changed files with 49 additions and 299 deletions

View File

@ -59,7 +59,7 @@
/** This hash function is not the most optimal, but UniquesHashSet states counted with it, /** 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 struct UniquesHashSetDefaultHash
{ {

View File

@ -55,7 +55,7 @@ public:
/// 2) Determine the includes file from the config: <include_from>/path2/metrika.xml</include_from> /// 2) Determine the includes file from the config: <include_from>/path2/metrika.xml</include_from>
/// If this path is not configured, use /etc/metrika.xml /// If this path is not configured, use /etc/metrika.xml
/// 3) Replace elements matching the "<foo incl="bar"/>" pattern with /// 3) Replace elements matching the "<foo incl="bar"/>" pattern with
/// "<foo>contents of the yandex/bar element in metrika.xml</foo>" /// "<foo>contents of the clickhouse/bar element in metrika.xml</foo>"
/// 4) If zk_node_cache is non-NULL, replace elements matching the "<foo from_zk="/bar">" pattern with /// 4) If zk_node_cache is non-NULL, replace elements matching the "<foo from_zk="/bar">" pattern with
/// "<foo>contents of the /bar ZooKeeper node</foo>". /// "<foo>contents of the /bar ZooKeeper node</foo>".
/// If has_zk_includes is non-NULL and there are such elements, set has_zk_includes to true. /// If has_zk_includes is non-NULL and there are such elements, set has_zk_includes to true.
@ -137,4 +137,3 @@ private:
}; };
} }

View File

@ -13,7 +13,7 @@
* *
* Example: when we do aggregation by the visitor ID, the performance increase is more than 5 times. * Example: when we do aggregation by the visitor ID, the performance increase is more than 5 times.
* This is because of following reasons: * 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; * - 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; * - 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, * - we are using open-addressing linear probing hash tables that are most critical to hash function quality,

View File

@ -28,7 +28,7 @@ void OptimizedRegularExpressionImpl<thread_safe>::analyze(
* in which all metacharacters are escaped, * in which all metacharacters are escaped,
* and also if there are no '|' outside the brackets, * and also if there are no '|' outside the brackets,
* and also avoid substrings of the form `http://` or `www` and some other * 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 * begin = regexp.data();
const char * pos = begin; const char * pos = begin;

View File

@ -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. # in places when no dependency on whole "dbms" library is possible.
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")

View File

@ -45,7 +45,7 @@ struct ZooKeeperResponse : virtual Response
using ZooKeeperResponsePtr = std::shared_ptr<ZooKeeperResponse>; using ZooKeeperResponsePtr = std::shared_ptr<ZooKeeperResponse>;
/// Exposed in header file for Yandex.Metrica code. /// Exposed in header file for some external code.
struct ZooKeeperRequest : virtual Request struct ZooKeeperRequest : virtual Request
{ {
XID xid = 0; XID xid = 0;

View File

@ -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) add_executable(zkutil_test_async zkutil_test_async.cpp)
target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) 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) add_executable (zookeeper_impl zookeeper_impl.cpp)
target_link_libraries (zookeeper_impl PRIVATE clickhouse_common_zookeeper_no_log) target_link_libraries (zookeeper_impl PRIVATE clickhouse_common_zookeeper_no_log)

View File

@ -1,66 +0,0 @@
#include <Common/Config/ConfigProcessor.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Poco/Event.h>
#include <iostream>
/// 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] << " <zookeeper_config> <number_of_watches>" << 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<Poco::Event>();
/// 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<std::thread> 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;
}
}

View File

@ -10,10 +10,10 @@
/** 'mysqlxx' - very simple library for replacement of 'mysql++' library. /** '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++': * There are the following shortcomings of 'mysql++':
* 1. Too rich functionality: most of it is not used. * 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: * 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, * And for the sake of simplicity, some functions work only with certain assumptions,
* or with slightly different semantic than in mysql++. * or with slightly different semantic than in mysql++.
* And we don't care about cross-platform usage of mysqlxx. * 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, * 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". * because it is developed from the principle - "everything that we don't need is not implemented".

View File

@ -13,8 +13,8 @@ namespace DB
* Otherwise, an exception is thrown. * Otherwise, an exception is thrown.
* *
* Examples: * Examples:
* yandex.ru - returns "yandex.ru" and default_port * clickhouse.com - returns "clickhouse.com" and default_port
* yandex.ru:80 - returns "yandex.ru" and 80 * 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. * [2a02:6b8:a::a]:80 - returns [2a02:6b8:a::a] and 80; note that square brackets remain in returned host.
*/ */
std::pair<std::string, UInt16> parseAddress(const std::string & str, UInt16 default_port); std::pair<std::string, UInt16> parseAddress(const std::string & str, UInt16 default_port);

View File

@ -539,7 +539,7 @@ struct ToStartOfFifteenMinutesImpl
using FactorTransform = ZeroTransform; 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 struct TimeSlotImpl
{ {
static constexpr auto name = "timeSlot"; static constexpr auto name = "timeSlot";

View File

@ -40,7 +40,7 @@ public:
registerFunction(name, &Function::create, case_sensitiveness); 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<std::string> getAllNames() const; std::vector<std::string> getAllNames() const;
bool has(const std::string & name) const; bool has(const std::string & name) const;

View File

@ -10,7 +10,7 @@
#include "config_functions.h" #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. * Enabled by default.
*/ */
#if USE_FASTOPS #if USE_FASTOPS

View File

@ -33,7 +33,7 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
/** Functions using Yandex.Metrica dictionaries /** Functions using deprecated dictionaries
* - dictionaries of regions, operating systems, search engines. * - dictionaries of regions, operating systems, search engines.
* *
* Climb up the tree to a certain level. * Climb up the tree to a certain level.

View File

@ -195,7 +195,7 @@ public:
* that is the initiator of a distributed query, * 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. * in the case when the function will be invoked for real data only at the remote servers.
* This feature is controversial and implemented specially * 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) if (input_rows_count == 0)
return result_type->createColumn(); return result_type->createColumn();

View File

@ -12,7 +12,7 @@
/** Functions for retrieving "visit parameters". /** 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. * These functions are applicable to almost any JSONs.
* Implemented via templates from FunctionsStringSearch.h. * Implemented via templates from FunctionsStringSearch.h.
* *

View File

@ -45,7 +45,7 @@ namespace DB
* Remove specified parameter from URL. * Remove specified parameter from URL.
* cutURLParameter(URL, name) * 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) * URLHierarchy(URL)
*/ */

View File

@ -15,7 +15,7 @@
// for better debug: #include <Core/iostream_debug_helpers.h> // for better debug: #include <Core/iostream_debug_helpers.h>
/** The function will enumerate distinct values of the passed multidimensional arrays looking inside at the specified depths. /** 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] * arrayEnumerateUniqRanked(['hello', 'world', 'hello']) = [1, 1, 2]
* - it returns similar structured array containing number of occurrence of the corresponding value. * - it returns similar structured array containing number of occurrence of the corresponding value.

View File

@ -718,9 +718,7 @@ private:
/** /**
* Catches arguments of type LowCardinality(T) (left) and U (right). * Catches arguments of type LowCardinality(T) (left) and U (right).
* *
* The perftests * The perftests showed that the amount of action needed to convert the non-constant right argument to the index column
* 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
* (similar to the left one's) is significantly higher than converting the array itself to an ordinary 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 * So, in terms of performance it's more optimal to fall back to default implementation and catch only constant

View File

@ -53,7 +53,7 @@
* *
* Usage example: * 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. * - ClickHouse has embedded web browser.
*/ */

View File

@ -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')]. * 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 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. * But this function was adopted by wider audience.
*/ */

View File

@ -343,7 +343,7 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
&& decoded_char != '"' && decoded_char != '"'
&& decoded_char != '`' /// MySQL style identifiers && decoded_char != '`' /// MySQL style identifiers
&& decoded_char != '/' /// JavaScript in HTML && decoded_char != '/' /// JavaScript in HTML
&& decoded_char != '=' /// Yandex's TSKV && decoded_char != '=' /// TSKV format invented somewhere
&& !isControlASCII(decoded_char)) && !isControlASCII(decoded_char))
{ {
s.push_back('\\'); s.push_back('\\');

View File

@ -16,36 +16,36 @@ class S3UriTest : public testing::TestWithParam<std::string>
TEST(S3UriTest, validPatterns) TEST(S3UriTest, validPatterns)
{ {
{ {
S3::URI uri(Poco::URI("https://jokserfn.s3.yandexcloud.net/")); S3::URI uri(Poco::URI("https://jokserfn.s3.amazonaws.com/"));
ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint);
ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("jokserfn", uri.bucket);
ASSERT_EQ("", uri.key); ASSERT_EQ("", uri.key);
ASSERT_EQ(true, uri.is_virtual_hosted_style); ASSERT_EQ(true, uri.is_virtual_hosted_style);
} }
{ {
S3::URI uri(Poco::URI("https://s3.yandexcloud.net/jokserfn/")); S3::URI uri(Poco::URI("https://s3.amazonaws.com/jokserfn/"));
ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint);
ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("jokserfn", uri.bucket);
ASSERT_EQ("", uri.key); ASSERT_EQ("", uri.key);
ASSERT_EQ(false, uri.is_virtual_hosted_style); ASSERT_EQ(false, uri.is_virtual_hosted_style);
} }
{ {
S3::URI uri(Poco::URI("https://yandexcloud.net/bucket/")); S3::URI uri(Poco::URI("https://amazonaws.com/bucket/"));
ASSERT_EQ("https://yandexcloud.net", uri.endpoint); ASSERT_EQ("https://amazonaws.com", uri.endpoint);
ASSERT_EQ("bucket", uri.bucket); ASSERT_EQ("bucket", uri.bucket);
ASSERT_EQ("", uri.key); ASSERT_EQ("", uri.key);
ASSERT_EQ(false, uri.is_virtual_hosted_style); ASSERT_EQ(false, uri.is_virtual_hosted_style);
} }
{ {
S3::URI uri(Poco::URI("https://jokserfn.s3.yandexcloud.net/data")); S3::URI uri(Poco::URI("https://jokserfn.s3.amazonaws.com/data"));
ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint);
ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("jokserfn", uri.bucket);
ASSERT_EQ("data", uri.key); ASSERT_EQ("data", uri.key);
ASSERT_EQ(true, uri.is_virtual_hosted_style); ASSERT_EQ(true, uri.is_virtual_hosted_style);
} }
{ {
S3::URI uri(Poco::URI("https://storage.yandexcloud.net/jokserfn/data")); S3::URI uri(Poco::URI("https://storage.amazonaws.com/jokserfn/data"));
ASSERT_EQ("https://storage.yandexcloud.net", uri.endpoint); ASSERT_EQ("https://storage.amazonaws.com", uri.endpoint);
ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("jokserfn", uri.bucket);
ASSERT_EQ("data", uri.key); ASSERT_EQ("data", uri.key);
ASSERT_EQ(false, uri.is_virtual_hosted_style); ASSERT_EQ(false, uri.is_virtual_hosted_style);
@ -97,13 +97,13 @@ INSTANTIATE_TEST_SUITE_P(
S3UriTest, S3UriTest,
testing::Values( testing::Values(
"https:///", "https:///",
"https://.s3.yandexcloud.net/key", "https://.s3.amazonaws.com/key",
"https://s3.yandexcloud.net/key", "https://s3.amazonaws.com/key",
"https://jokserfn.s3yandexcloud.net/key", "https://jokserfn.s3amazonaws.com/key",
"https://s3.yandexcloud.net//", "https://s3.amazonaws.com//",
"https://yandexcloud.net/", "https://amazonaws.com/",
"https://yandexcloud.net//", "https://amazonaws.com//",
"https://yandexcloud.net//key")); "https://amazonaws.com//key"));
} }

View File

@ -28,7 +28,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v
} }
/// cast of numeric constant in condition to UInt8 /// 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. /// We should allow any constant condition (or maybe remove this optimization completely) later.
if (const auto * function = condition->as<ASTFunction>()) if (const auto * function = condition->as<ASTFunction>())
{ {

View File

@ -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. * 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. * 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) static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const IAST::FormatSettings & settings, const char * metacharacters)
{ {

View File

@ -1727,7 +1727,7 @@ const char * ParserAlias::restricted_keywords[] =
"NOT", "NOT",
"OFFSET", "OFFSET",
"ON", "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", "ORDER",
"PREWHERE", "PREWHERE",
"RIGHT", "RIGHT",

View File

@ -52,6 +52,7 @@ void JSONAsRowInputFormat::readSuffix()
{ {
assertChar(']', *buf); assertChar(']', *buf);
skipWhitespaceIfAny(*buf); skipWhitespaceIfAny(*buf);
data_in_square_brackets = false;
} }
if (!buf->eof() && *buf->position() == ';') if (!buf->eof() && *buf->position() == ';')
{ {

View File

@ -149,7 +149,7 @@ TEST(GraphiteTest, testSelectPattern)
using namespace std::literals; using namespace std::literals;
std::string std::string
xml(R"END(<yandex> xml(R"END(<clickhouse>
<graphite_rollup> <graphite_rollup>
<pattern> <pattern>
<regexp>\.sum$</regexp> <regexp>\.sum$</regexp>
@ -210,7 +210,7 @@ TEST(GraphiteTest, testSelectPattern)
</retention> </retention>
</default> </default>
</graphite_rollup> </graphite_rollup>
</yandex> </clickhouse>
)END"); )END");
// Retentions must be ordered by 'age' descending. // Retentions must be ordered by 'age' descending.
@ -370,7 +370,7 @@ TEST(GraphiteTest, testSelectPatternTyped)
using namespace std::literals; using namespace std::literals;
std::string std::string
xml(R"END(<yandex> xml(R"END(<clickhouse>
<graphite_rollup> <graphite_rollup>
<pattern> <pattern>
<rule_type>plain</rule_type> <rule_type>plain</rule_type>
@ -488,7 +488,7 @@ TEST(GraphiteTest, testSelectPatternTyped)
</retention> </retention>
</default> </default>
</graphite_rollup> </graphite_rollup>
</yandex> </clickhouse>
)END"); )END");
// Retentions must be ordered by 'age' descending. // Retentions must be ordered by 'age' descending.

View File

@ -12,7 +12,7 @@ class ActionsDAG;
/** Executes a certain expression over the block. /** Executes a certain expression over the block.
* The expression consists of column identifiers from the block, constants, common functions. * 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. * The expression processes each row independently of the others.
*/ */
class ExpressionTransform final : public ISimpleTransform class ExpressionTransform final : public ISimpleTransform

View File

@ -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) add_executable (merge_selector merge_selector.cpp)
target_link_libraries (merge_selector PRIVATE dbms) 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) 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) 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
)

View File

@ -1,35 +0,0 @@
#include <unistd.h>
#include <iostream>
#include <Common/Exception.h>
#include <Common/filesystemHelpers.h>
#include <filesystem>
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;
}

View File

@ -1,131 +0,0 @@
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <boost/program_options.hpp>
#include <list>
#include <iostream>
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<std::string>()->required(),
"addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181")
("path,p", boost::program_options::value<std::string>()->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>());
std::string initial_path = options.at("path").as<std::string>();
struct Node
{
Node(
std::string path_,
std::future<Coordination::GetResponse> get_future_,
std::future<Coordination::ListResponse> 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<Coordination::GetResponse> get_future;
std::future<Coordination::ListResponse> children_future;
Node * parent = nullptr;
std::future<Coordination::MultiResponse> set_future;
};
std::list<Node> 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;
}