Merge branch 'master' into libcxx14

This commit is contained in:
mergify[bot] 2022-03-24 17:39:01 +00:00 committed by GitHub
commit 0b95348215
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
97 changed files with 1581 additions and 919 deletions

2
contrib/libxml2 vendored

@ -1 +1 @@
Subproject commit 18890f471c420411aa3c989e104d090966ec9dbf
Subproject commit a075d256fd9ff15590b86d981b75a50ead124fca

View File

@ -15,7 +15,7 @@
```
┌─name─────────────────────┬─is_aggregate─┬─case_insensitive─┬─alias_to─┐
│ sumburConsistentHash │ 0 │ 0 │ │
yandexConsistentHash │ 0 │ 0 │ │
kostikConsistentHash │ 0 │ 0 │ │
│ demangle │ 0 │ 0 │ │
│ addressToLine │ 0 │ 0 │ │
│ JSONExtractRaw │ 0 │ 0 │ │

View File

@ -29,15 +29,15 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
time_t decrease_error_period_,
size_t max_error_cap_)
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover"))
, default_load_balancing(load_balancing)
, get_priority_load_balancing(load_balancing)
{
const std::string & local_hostname = getFQDNOrHostName();
hostname_differences.resize(nested_pools.size());
get_priority_load_balancing.hostname_differences.resize(nested_pools.size());
for (size_t i = 0; i < nested_pools.size(); ++i)
{
ConnectionPool & connection_pool = dynamic_cast<ConnectionPool &>(*nested_pools[i]);
hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost());
get_priority_load_balancing.hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost());
}
}
@ -51,36 +51,15 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts
};
size_t offset = 0;
LoadBalancing load_balancing = get_priority_load_balancing.load_balancing;
if (settings)
offset = settings->load_balancing_first_offset % nested_pools.size();
GetPriorityFunc get_priority;
switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing)
{
case LoadBalancing::NEAREST_HOSTNAME:
get_priority = [&](size_t i) { return hostname_differences[i]; };
break;
case LoadBalancing::IN_ORDER:
get_priority = [](size_t i) { return i; };
break;
case LoadBalancing::RANDOM:
break;
case LoadBalancing::FIRST_OR_RANDOM:
get_priority = [offset](size_t i) -> size_t { return i != offset; };
break;
case LoadBalancing::ROUND_ROBIN:
if (last_used >= nested_pools.size())
last_used = 0;
++last_used;
/* Consider nested_pools.size() equals to 5
* last_used = 1 -> get_priority: 0 1 2 3 4
* last_used = 2 -> get_priority: 4 0 1 2 3
* last_used = 3 -> get_priority: 4 3 0 1 2
* ...
* */
get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; };
break;
offset = settings->load_balancing_first_offset % nested_pools.size();
load_balancing = LoadBalancing(settings->load_balancing);
}
GetPriorityFunc get_priority = get_priority_load_balancing.getPriorityFunc(load_balancing, offset, nested_pools.size());
UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0;
bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true;
@ -173,38 +152,14 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings * settings)
{
size_t offset = 0;
LoadBalancing load_balancing = get_priority_load_balancing.load_balancing;
if (settings)
offset = settings->load_balancing_first_offset % nested_pools.size();
GetPriorityFunc get_priority;
switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing)
{
case LoadBalancing::NEAREST_HOSTNAME:
get_priority = [&](size_t i) { return hostname_differences[i]; };
break;
case LoadBalancing::IN_ORDER:
get_priority = [](size_t i) { return i; };
break;
case LoadBalancing::RANDOM:
break;
case LoadBalancing::FIRST_OR_RANDOM:
get_priority = [offset](size_t i) -> size_t { return i != offset; };
break;
case LoadBalancing::ROUND_ROBIN:
if (last_used >= nested_pools.size())
last_used = 0;
++last_used;
/* Consider nested_pools.size() equals to 5
* last_used = 1 -> get_priority: 0 1 2 3 4
* last_used = 2 -> get_priority: 5 0 1 2 3
* last_used = 3 -> get_priority: 5 4 0 1 2
* ...
* */
get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; };
break;
offset = settings->load_balancing_first_offset % nested_pools.size();
load_balancing = LoadBalancing(settings->load_balancing);
}
return get_priority;
return get_priority_load_balancing.getPriorityFunc(load_balancing, offset, nested_pools.size());
}
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyImpl(

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/PoolWithFailoverBase.h>
#include <Common/GetPriorityForLoadBalancing.h>
#include <Client/ConnectionPool.h>
#include <chrono>
@ -109,9 +110,7 @@ private:
GetPriorityFunc makeGetPriorityFunc(const Settings * settings);
std::vector<size_t> hostname_differences; /// Distances from name of this host to the names of hosts of pools.
size_t last_used = 0; /// Last used for round_robin policy.
LoadBalancing default_load_balancing;
GetPriorityForLoadBalancing get_priority_load_balancing;
};
using ConnectionPoolWithFailoverPtr = std::shared_ptr<ConnectionPoolWithFailover>;

View File

@ -83,11 +83,20 @@ size_t extractMaskNumericImpl(
const PaddedPODArray<UInt8> * null_bytemap,
PaddedPODArray<UInt8> * nulls)
{
if constexpr (!column_is_short)
{
if (data.size() != mask.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of a full data column is not equal to the size of a mask");
}
size_t ones_count = 0;
size_t data_index = 0;
size_t mask_size = mask.size();
for (size_t i = 0; i != mask_size; ++i)
size_t mask_size = mask.size();
size_t data_size = data.size();
size_t i = 0;
for (; i != mask_size && data_index != data_size; ++i)
{
// Change mask only where value is 1.
if (!mask[i])
@ -120,6 +129,13 @@ size_t extractMaskNumericImpl(
mask[i] = value;
}
if constexpr (column_is_short)
{
if (data_index != data_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of a short column is not equal to the number of ones in a mask");
}
return ones_count;
}

View File

@ -0,0 +1,49 @@
#include <Common/GetPriorityForLoadBalancing.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
std::function<size_t(size_t index)> GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const
{
std::function<size_t(size_t index)> get_priority;
switch (load_balance)
{
case LoadBalancing::NEAREST_HOSTNAME:
if (hostname_differences.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_differences is not initialized");
get_priority = [&](size_t i) { return hostname_differences[i]; };
break;
case LoadBalancing::IN_ORDER:
get_priority = [](size_t i) { return i; };
break;
case LoadBalancing::RANDOM:
break;
case LoadBalancing::FIRST_OR_RANDOM:
get_priority = [offset](size_t i) -> size_t { return i != offset; };
break;
case LoadBalancing::ROUND_ROBIN:
if (last_used >= pool_size)
last_used = 0;
++last_used;
/* Consider pool_size equals to 5
* last_used = 1 -> get_priority: 0 1 2 3 4
* last_used = 2 -> get_priority: 4 0 1 2 3
* last_used = 3 -> get_priority: 4 3 0 1 2
* ...
* */
get_priority = [&](size_t i)
{
++i;
return i < last_used ? pool_size - i : i - last_used;
};
break;
}
return get_priority;
}
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <Core/SettingsEnums.h>
namespace DB
{
class GetPriorityForLoadBalancing
{
public:
GetPriorityForLoadBalancing(LoadBalancing load_balancing_) : load_balancing(load_balancing_) {}
GetPriorityForLoadBalancing(){}
bool operator == (const GetPriorityForLoadBalancing & other) const
{
return load_balancing == other.load_balancing && hostname_differences == other.hostname_differences;
}
bool operator != (const GetPriorityForLoadBalancing & other) const
{
return !(*this == other);
}
std::function<size_t(size_t index)> getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const;
std::vector<size_t> hostname_differences; /// Distances from name of this host to the names of hosts of pools.
LoadBalancing load_balancing = LoadBalancing::RANDOM;
private:
mutable size_t last_used = 0; /// Last used for round_robin policy.
};
}

View File

@ -22,7 +22,6 @@ target_link_libraries (clickhouse_common_zookeeper_no_log
PRIVATE
string_utils
)
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()

View File

@ -5,15 +5,15 @@
#include <functional>
#include <filesystem>
#include <pcg-random/pcg_random.hpp>
#include <base/logger_useful.h>
#include <base/find_symbols.h>
#include <Common/randomSeed.h>
#include <base/getFQDNOrHostName.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#include <Common/isLocalAddress.h>
#include <Poco/Net/NetException.h>
#include <Poco/Net/DNS.h>
#define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000
@ -48,7 +48,7 @@ static void check(Coordination::Error code, const std::string & path)
void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_,
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_)
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const GetPriorityForLoadBalancing & get_priority_load_balancing_)
{
log = &Poco::Logger::get("ZooKeeper");
hosts = hosts_;
@ -57,6 +57,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_
operation_timeout_ms = operation_timeout_ms_;
chroot = chroot_;
implementation = implementation_;
get_priority_load_balancing = get_priority_load_balancing_;
if (implementation == "zookeeper")
{
@ -66,14 +67,13 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_
Coordination::ZooKeeper::Nodes nodes;
nodes.reserve(hosts.size());
Strings shuffled_hosts = hosts;
/// Shuffle the hosts to distribute the load among ZooKeeper nodes.
pcg64 generator(randomSeed());
std::shuffle(shuffled_hosts.begin(), shuffled_hosts.end(), generator);
std::vector<ShuffleHost> shuffled_hosts = shuffleHosts();
bool dns_error = false;
for (auto & host_string : shuffled_hosts)
for (auto & host : shuffled_hosts)
{
auto & host_string = host.host;
try
{
bool secure = bool(startsWith(host_string, "secure://"));
@ -81,6 +81,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_
if (secure)
host_string.erase(0, strlen("secure://"));
LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString());
nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure});
}
catch (const Poco::Net::HostNotFoundException & e)
@ -154,23 +155,47 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_
}
}
std::vector<ShuffleHost> ZooKeeper::shuffleHosts() const
{
std::function<size_t(size_t index)> get_priority = get_priority_load_balancing.getPriorityFunc(get_priority_load_balancing.load_balancing, 0, hosts.size());
std::vector<ShuffleHost> shuffle_hosts;
for (size_t i = 0; i < hosts.size(); ++i)
{
ShuffleHost shuffle_host;
shuffle_host.host = hosts[i];
if (get_priority)
shuffle_host.priority = get_priority(i);
shuffle_host.randomize();
shuffle_hosts.emplace_back(shuffle_host);
}
std::sort(
shuffle_hosts.begin(), shuffle_hosts.end(),
[](const ShuffleHost & lhs, const ShuffleHost & rhs)
{
return ShuffleHost::compare(lhs, rhs);
});
return shuffle_hosts;
}
ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_,
int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_,
std::shared_ptr<DB::ZooKeeperLog> zk_log_)
std::shared_ptr<DB::ZooKeeperLog> zk_log_, const GetPriorityForLoadBalancing & get_priority_load_balancing_)
{
zk_log = std::move(zk_log_);
Strings hosts_strings;
splitInto<','>(hosts_strings, hosts_string);
init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_);
init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, get_priority_load_balancing_);
}
ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_,
int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_,
std::shared_ptr<DB::ZooKeeperLog> zk_log_)
std::shared_ptr<DB::ZooKeeperLog> zk_log_, const GetPriorityForLoadBalancing & get_priority_load_balancing_)
{
zk_log = std::move(zk_log_);
init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_);
init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, get_priority_load_balancing_);
}
struct ZooKeeperArgs
@ -213,6 +238,15 @@ struct ZooKeeperArgs
{
implementation = config.getString(config_name + "." + key);
}
else if (key == "zookeeper_load_balancing")
{
String load_balancing_str = config.getString(config_name + "." + key);
/// Use magic_enum to avoid dependency from dbms (`SettingFieldLoadBalancingTraits::fromString(...)`)
auto load_balancing = magic_enum::enum_cast<DB::LoadBalancing>(Poco::toUpper(load_balancing_str));
if (!load_balancing)
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknown load balancing: {}", load_balancing_str);
get_priority_load_balancing.load_balancing = *load_balancing;
}
else
throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS);
}
@ -224,6 +258,15 @@ struct ZooKeeperArgs
if (chroot.back() == '/')
chroot.pop_back();
}
/// init get_priority_load_balancing
get_priority_load_balancing.hostname_differences.resize(hosts.size());
const String & local_hostname = getFQDNOrHostName();
for (size_t i = 0; i < hosts.size(); ++i)
{
const String & node_host = hosts[i].substr(0, hosts[i].find_last_of(':'));
get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, node_host);
}
}
Strings hosts;
@ -232,13 +275,14 @@ struct ZooKeeperArgs
int operation_timeout_ms;
std::string chroot;
std::string implementation;
GetPriorityForLoadBalancing get_priority_load_balancing;
};
ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, std::shared_ptr<DB::ZooKeeperLog> zk_log_)
: zk_log(std::move(zk_log_))
{
ZooKeeperArgs args(config, config_name);
init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot);
init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.get_priority_load_balancing);
}
bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) const
@ -249,8 +293,11 @@ bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config,
if (args.implementation == implementation && implementation == "testkeeper")
return false;
return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot)
!= std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot);
if (args.get_priority_load_balancing != get_priority_load_balancing)
return true;
return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.get_priority_load_balancing)
!= std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, args.get_priority_load_balancing);
}
@ -757,7 +804,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition &
ZooKeeperPtr ZooKeeper::startNewSession() const
{
return std::make_shared<ZooKeeper>(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log);
return std::make_shared<ZooKeeper>(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log, get_priority_load_balancing);
}

View File

@ -13,7 +13,10 @@
#include <Common/Stopwatch.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h>
#include <Common/GetPriorityForLoadBalancing.h>
#include <Common/thread_local_rng.h>
#include <unistd.h>
#include <random>
namespace ProfileEvents
@ -37,6 +40,25 @@ namespace zkutil
/// Preferred size of multi() command (in number of ops)
constexpr size_t MULTI_BATCH_SIZE = 100;
struct ShuffleHost
{
String host;
Int64 priority = 0;
UInt32 random = 0;
void randomize()
{
random = thread_local_rng();
}
static bool compare(const ShuffleHost & lhs, const ShuffleHost & rhs)
{
return std::forward_as_tuple(lhs.priority, lhs.random)
< std::forward_as_tuple(rhs.priority, rhs.random);
}
};
using GetPriorityForLoadBalancing = DB::GetPriorityForLoadBalancing;
/// ZooKeeper session. The interface is substantially different from the usual libzookeeper API.
///
@ -58,14 +80,16 @@ public:
int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS,
const std::string & chroot_ = "",
const std::string & implementation_ = "zookeeper",
std::shared_ptr<DB::ZooKeeperLog> zk_log_ = nullptr);
std::shared_ptr<DB::ZooKeeperLog> zk_log_ = nullptr,
const GetPriorityForLoadBalancing & get_priority_load_balancing_ = {});
explicit ZooKeeper(const Strings & hosts_, const std::string & identity_ = "",
int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS,
int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS,
const std::string & chroot_ = "",
const std::string & implementation_ = "zookeeper",
std::shared_ptr<DB::ZooKeeperLog> zk_log_ = nullptr);
std::shared_ptr<DB::ZooKeeperLog> zk_log_ = nullptr,
const GetPriorityForLoadBalancing & get_priority_load_balancing_ = {});
/** Config of the form:
<zookeeper>
@ -91,6 +115,8 @@ public:
*/
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, std::shared_ptr<DB::ZooKeeperLog> zk_log_);
std::vector<ShuffleHost> shuffleHosts() const;
/// Creates a new session with the same parameters. This method can be used for reconnecting
/// after the session has expired.
/// This object remains unchanged, and the new session is returned.
@ -284,7 +310,7 @@ private:
friend class EphemeralNodeHolder;
void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_,
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_);
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const GetPriorityForLoadBalancing & get_priority_load_balancing_);
/// The following methods don't any throw exceptions but return error codes.
Coordination::Error createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created);
@ -311,6 +337,8 @@ private:
Poco::Logger * log = nullptr;
std::shared_ptr<DB::ZooKeeperLog> zk_log;
GetPriorityForLoadBalancing get_priority_load_balancing;
AtomicStopwatch session_uptime;
};

View File

@ -451,7 +451,7 @@ void ZooKeeper::connect(
}
else
{
LOG_TEST(log, "Connected to ZooKeeper at {} with session_id {}", socket.peerAddress().toString(), session_id);
LOG_TEST(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str());
}
}

View File

@ -124,6 +124,7 @@ bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_
size_t getHostNameDifference(const std::string & local_hostname, const std::string & host)
{
/// FIXME should we replace it with Levenstein distance? (we already have it in NamePrompter)
size_t hostname_difference = 0;
for (size_t i = 0; i < std::min(local_hostname.length(), host.length()); ++i)
if (local_hostname[i] != host[i])

View File

@ -149,4 +149,5 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation , ErrorCodes::BAD_ARGUMENTS,
{"str", FormatSettings::MsgPackUUIDRepresentation::STR},
{"ext", FormatSettings::MsgPackUUIDRepresentation::EXT}})
}

View File

@ -33,7 +33,7 @@ DataTypePtr DataTypeFactory::get(const String & full_name) const
/// Value 315 is known to cause stack overflow in some test configurations (debug build, sanitizers)
/// let's make the threshold significantly lower.
/// It is impractical for user to have complex data types with this depth.
static constexpr size_t data_type_max_parse_depth = 150;
static constexpr size_t data_type_max_parse_depth = 200;
ParserDataType parser;
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0, data_type_max_parse_depth);

View File

@ -20,6 +20,7 @@
#include <Common/getRandomASCIIString.h>
#include <Interpreters/Context.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <IO/ReadBufferFromS3.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
@ -264,32 +265,6 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
LOG_TRACE(log, "{} to file by path: {}. S3 path: {}",
mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), remote_fs_root_path + blob_name);
ScheduleFunc schedule = [pool = &getThreadPoolWriter(), thread_group = CurrentThread::getGroup()](auto callback)
{
pool->scheduleOrThrow([callback = std::move(callback), thread_group]()
{
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
/// After we detached from the thread_group, parent for memory_tracker inside ThreadStatus will be reset to it's parent.
/// Typically, it may be changes from Process to User.
/// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed.
/// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well.
/// When, finally, we destroy the thread (and the ThreadStatus),
/// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,\
/// and by this time user-level memory tracker may be already destroyed.
///
/// As a work-around, reset memory tracker to total, which is always alive.
CurrentThread::get().memory_tracker.setParent(&total_memory_tracker);
);
callback();
});
};
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
settings->client,
bucket,
@ -299,7 +274,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
settings->s3_upload_part_size_multiply_parts_count_threshold,
settings->s3_max_single_part_upload_size,
std::move(object_metadata),
buf_size, std::move(schedule));
buf_size, threadPoolCallbackRunner(getThreadPoolWriter()));
auto create_metadata_callback = [this, path, blob_name, mode] (size_t count)
{

View File

@ -7,9 +7,9 @@ namespace DB
{
/// An O(1) time and space consistent hash algorithm by Konstantin Oblakov
struct YandexConsistentHashImpl
struct KostikConsistentHashImpl
{
static constexpr auto name = "yandexConsistentHash";
static constexpr auto name = "kostikConsistentHash";
using HashType = UInt64;
/// Actually it supports UInt64, but it is efficient only if n <= 32768
@ -23,12 +23,12 @@ struct YandexConsistentHashImpl
}
};
using FunctionYandexConsistentHash = FunctionConsistentHashImpl<YandexConsistentHashImpl>;
using FunctionKostikConsistentHash = FunctionConsistentHashImpl<KostikConsistentHashImpl>;
void registerFunctionYandexConsistentHash(FunctionFactory & factory)
void registerFunctionKostikConsistentHash(FunctionFactory & factory)
{
factory.registerFunction<FunctionYandexConsistentHash>();
factory.registerFunction<FunctionKostikConsistentHash>();
factory.registerAlias("yandexConsistentHash", "kostikConsistentHash");
}
}

View File

@ -2,12 +2,12 @@ namespace DB
{
class FunctionFactory;
void registerFunctionYandexConsistentHash(FunctionFactory & factory);
void registerFunctionKostikConsistentHash(FunctionFactory & factory);
void registerFunctionJumpConsistentHash(FunctionFactory & factory);
void registerFunctionsConsistentHashing(FunctionFactory & factory)
{
registerFunctionYandexConsistentHash(factory);
registerFunctionKostikConsistentHash(factory);
registerFunctionJumpConsistentHash(factory);
}

View File

@ -372,8 +372,8 @@ SetPtr makeExplicitSet(
element_type = low_cardinality_type->getDictionaryType();
auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types);
if (prepared_sets.count(set_key))
return prepared_sets.at(set_key); /// Already prepared.
if (auto it = prepared_sets.find(set_key); it != prepared_sets.end())
return it->second; /// Already prepared.
Block block;
const auto & right_arg_func = std::dynamic_pointer_cast<ASTFunction>(right_arg);
@ -388,7 +388,7 @@ SetPtr makeExplicitSet(
set->insertFromBlock(block.getColumnsWithTypeAndName());
set->finishInsert();
prepared_sets[set_key] = set;
prepared_sets.emplace(set_key, set);
return set;
}
@ -707,7 +707,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat
if (tid != 0)
tuple_ast = tuple_ast->clone();
auto literal = std::make_shared<ASTLiteral>(UInt64(++tid));
auto literal = std::make_shared<ASTLiteral>(UInt64{++tid});
visit(*literal, literal, data);
auto func = makeASTFunction("tupleElement", tuple_ast, literal);
@ -814,14 +814,13 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
if (!data.only_consts)
{
/// We are in the part of the tree that we are not going to compute. You just need to define types.
/// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet".
/// Do not evaluate subquery and create sets. We replace "in*" function to "in*IgnoreSet".
auto argument_name = node.arguments->children.at(0)->getColumnName();
data.addFunction(
FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()),
{ argument_name, argument_name },
column_name);
FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()),
{argument_name, argument_name},
column_name);
}
return;
}
@ -1145,8 +1144,8 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
if (no_subqueries)
return {};
auto set_key = PreparedSetKey::forSubquery(*right_in_operand);
if (data.prepared_sets.count(set_key))
return data.prepared_sets.at(set_key);
if (auto it = data.prepared_sets.find(set_key); it != data.prepared_sets.end())
return it->second;
/// A special case is if the name of the table is specified on the right side of the IN statement,
/// and the table has the type Set (a previously prepared set).
@ -1160,7 +1159,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
if (storage_set)
{
data.prepared_sets[set_key] = storage_set->getSet();
data.prepared_sets.emplace(set_key, storage_set->getSet());
return storage_set->getSet();
}
}
@ -1174,7 +1173,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
/// If you already created a Set with the same subquery / table.
if (subquery_for_set.set)
{
data.prepared_sets[set_key] = subquery_for_set.set;
data.prepared_sets.emplace(set_key, subquery_for_set.set);
return subquery_for_set.set;
}
@ -1196,7 +1195,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
}
subquery_for_set.set = set;
data.prepared_sets[set_key] = set;
data.prepared_sets.emplace(set_key, set);
return set;
}
else

View File

@ -10,6 +10,7 @@
namespace DB
{
class ASTExpressionList;
class ASTFunction;
class ExpressionActions;
@ -89,10 +90,7 @@ struct ScopeStack : WithContext
void addColumn(ColumnWithTypeAndName column);
void addAlias(const std::string & name, std::string alias);
void addArrayJoin(const std::string & source_name, std::string result_name);
void addFunction(
const FunctionOverloadResolverPtr & function,
const Names & argument_names,
std::string result_name);
void addFunction(const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name);
ActionsDAGPtr popLevel();

View File

@ -350,6 +350,12 @@ void DDLWorker::scheduleTasks(bool reinitialized)
bool maybe_concurrently_deleting = task && !zookeeper->exists(fs::path(task->entry_path) / "active");
return task && !maybe_concurrently_deleting && !maybe_currently_processing;
}
else if (last_skipped_entry_name.has_value() && !queue_fully_loaded_after_initialization_debug_helper)
{
/// If connection was lost during queue loading
/// we may start processing from finished task (because we don't know yet that it's finished) and it's ok.
return false;
}
else
{
/// Return true if entry should not be scheduled.
@ -365,7 +371,11 @@ void DDLWorker::scheduleTasks(bool reinitialized)
String reason;
auto task = initAndCheckTask(entry_name, reason, zookeeper);
if (!task)
if (task)
{
queue_fully_loaded_after_initialization_debug_helper = true;
}
else
{
LOG_DEBUG(log, "Will not execute task {}: {}", entry_name, reason);
updateMaxDDLEntryID(entry_name);

View File

@ -131,6 +131,9 @@ protected:
std::optional<String> first_failed_task_name;
std::list<DDLTaskPtr> current_tasks;
/// This flag is needed for debug assertions only
bool queue_fully_loaded_after_initialization_debug_helper = false;
Coordination::Stat queue_node_stat;
std::shared_ptr<Poco::Event> queue_updated_event = std::make_shared<Poco::Event>();
std::shared_ptr<Poco::Event> cleanup_event = std::make_shared<Poco::Event>();

View File

@ -259,7 +259,7 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & a
if (!array_join_expression_list)
return src_columns;
getRootActionsNoMakeSet(array_join_expression_list, true, actions, false);
getRootActionsNoMakeSet(array_join_expression_list, actions, false);
auto array_join = addMultipleArrayJoinAction(actions, is_array_join_left);
auto sample_columns = actions->getResultColumns();
@ -294,7 +294,7 @@ NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), true, actions, false);
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), actions, false);
auto sample_columns = actions->getNamesAndTypesList();
syntax->analyzed_join->addJoinedColumnsAndCorrectTypes(sample_columns, true);
actions = std::make_shared<ActionsDAG>(sample_columns);
@ -332,14 +332,14 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
{
NameSet unique_keys;
ASTs & group_asts = group_by_ast->children;
for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i)
for (ssize_t i = 0; i < static_cast<ssize_t>(group_asts.size()); ++i)
{
ssize_t size = group_asts.size();
if (getContext()->getSettingsRef().enable_positional_arguments)
replaceForPositionalArguments(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY);
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
getRootActionsNoMakeSet(group_asts[i], temp_actions, false);
const auto & column_name = group_asts[i]->getColumnName();
@ -405,8 +405,8 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global)
{
if (do_global)
{
GlobalSubqueriesVisitor::Data subqueries_data(getContext(), subquery_depth, isRemoteStorage(),
external_tables, subqueries_for_sets, has_global_subqueries);
GlobalSubqueriesVisitor::Data subqueries_data(
getContext(), subquery_depth, isRemoteStorage(), external_tables, subqueries_for_sets, has_global_subqueries);
GlobalSubqueriesVisitor(subqueries_data).visit(query);
}
}
@ -416,7 +416,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
{
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
if (prepared_sets.count(set_key))
if (prepared_sets.contains(set_key))
return; /// Already prepared.
if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name))
@ -509,33 +509,62 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
}
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts)
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts)
{
LogAST log;
ActionsVisitor::Data visitor_data(getContext(), settings.size_limits_for_set, subquery_depth,
sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets,
no_subqueries, false, only_consts, !isRemoteStorage());
ActionsVisitor::Data visitor_data(
getContext(),
settings.size_limits_for_set,
subquery_depth,
sourceColumns(),
std::move(actions),
prepared_sets,
subqueries_for_sets,
no_makeset_for_subqueries,
false /* no_makeset */,
only_consts,
!isRemoteStorage() /* create_source_for_in */);
ActionsVisitor(visitor_data, log.stream()).visit(ast);
actions = visitor_data.getActions();
}
void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts)
void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts)
{
LogAST log;
ActionsVisitor::Data visitor_data(getContext(), settings.size_limits_for_set, subquery_depth,
sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets,
no_subqueries, true, only_consts, !isRemoteStorage());
ActionsVisitor::Data visitor_data(
getContext(),
settings.size_limits_for_set,
subquery_depth,
sourceColumns(),
std::move(actions),
prepared_sets,
subqueries_for_sets,
true /* no_makeset_for_subqueries, no_makeset implies no_makeset_for_subqueries */,
true /* no_makeset */,
only_consts,
!isRemoteStorage() /* create_source_for_in */);
ActionsVisitor(visitor_data, log.stream()).visit(ast);
actions = visitor_data.getActions();
}
void ExpressionAnalyzer::getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts)
void ExpressionAnalyzer::getRootActionsForHaving(
const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts)
{
LogAST log;
ActionsVisitor::Data visitor_data(getContext(), settings.size_limits_for_set, subquery_depth,
sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets,
no_subqueries, false, only_consts, true);
ActionsVisitor::Data visitor_data(
getContext(),
settings.size_limits_for_set,
subquery_depth,
sourceColumns(),
std::move(actions),
prepared_sets,
subqueries_for_sets,
no_makeset_for_subqueries,
false /* no_makeset */,
only_consts,
true /* create_source_for_in */);
ActionsVisitor(visitor_data, log.stream()).visit(ast);
actions = visitor_data.getActions();
}
@ -547,7 +576,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr
{
AggregateDescription aggregate;
if (node->arguments)
getRootActionsNoMakeSet(node->arguments, true, actions);
getRootActionsNoMakeSet(node->arguments, actions);
aggregate.column_name = node->getColumnName();
@ -746,8 +775,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
// Requiring a constant reference to a shared pointer to non-const AST
// doesn't really look sane, but the visitor does indeed require it.
// Hence we clone the node (not very sane either, I know).
getRootActionsNoMakeSet(window_function.function_node->clone(),
true, actions);
getRootActionsNoMakeSet(window_function.function_node->clone(), actions);
const ASTs & arguments
= window_function.function_node->arguments->children;
@ -867,8 +895,7 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi
auto array_join = addMultipleArrayJoinAction(step.actions(), is_array_join_left);
before_array_join = chain.getLastActions();
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ArrayJoinStep>(
array_join, step.getResultColumns()));
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ArrayJoinStep>(array_join, step.getResultColumns()));
chain.addStep();
@ -1099,8 +1126,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
}
}
chain.steps.emplace_back(std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(
std::make_shared<ActionsDAG>(std::move(columns))));
chain.steps.emplace_back(
std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(std::make_shared<ActionsDAG>(std::move(columns))));
chain.steps.back()->additional_input = std::move(unused_source_columns);
chain.getLastActions();
chain.addStep();
@ -1210,8 +1237,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
// recursively together with (1b) as ASTFunction::window_definition.
if (getSelectQuery()->window())
{
getRootActionsNoMakeSet(getSelectQuery()->window(),
true /* no_subqueries */, step.actions());
getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions());
}
for (const auto & [_, w] : window_descriptions)
@ -1222,8 +1248,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
// definitions (1a).
// Requiring a constant reference to a shared pointer to non-const AST
// doesn't really look sane, but the visitor does indeed require it.
getRootActionsNoMakeSet(f.function_node->clone(),
true /* no_subqueries */, step.actions());
getRootActionsNoMakeSet(f.function_node->clone(), step.actions());
// (2b) Required function argument columns.
for (const auto & a : f.function_node->arguments->children)
@ -1456,7 +1481,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
alias = name;
result_columns.emplace_back(name, alias);
result_names.push_back(alias);
getRootActions(ast, false, actions_dag);
getRootActions(ast, false /* no_makeset_for_subqueries */, actions_dag);
}
if (add_aliases)
@ -1496,7 +1521,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAn
{
auto actions = std::make_shared<ActionsDAG>(constant_inputs);
getRootActions(query, true, actions, true);
getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */);
return std::make_shared<ExpressionActions>(actions, ExpressionActionsSettings::fromContext(getContext()));
}
@ -1513,13 +1538,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions()
}
ExpressionAnalysisResult::ExpressionAnalysisResult(
SelectQueryExpressionAnalyzer & query_analyzer,
const StorageMetadataPtr & metadata_snapshot,
bool first_stage_,
bool second_stage_,
bool only_types,
const FilterDAGInfoPtr & filter_info_,
const Block & source_header)
SelectQueryExpressionAnalyzer & query_analyzer,
const StorageMetadataPtr & metadata_snapshot,
bool first_stage_,
bool second_stage_,
bool only_types,
const FilterDAGInfoPtr & filter_info_,
const Block & source_header)
: first_stage(first_stage_)
, second_stage(second_stage_)
, need_aggregate(query_analyzer.hasAggregation())

View File

@ -172,15 +172,15 @@ protected:
ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool is_left) const;
void getRootActions(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
/** Similar to getRootActions but do not make sets when analyzing IN functions. It's used in
* analyzeAggregation which happens earlier than analyzing PREWHERE and WHERE. If we did, the
* prepared sets would not be applicable for MergeTree index optimization.
*/
void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActionsForHaving(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
* Create a set of columns aggregated_columns resulting after the aggregation, if any,

View File

@ -10,6 +10,7 @@
#include <Interpreters/interpretSubquery.h>
#include <Interpreters/SubqueryForSet.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
@ -17,7 +18,11 @@
#include <Parsers/IAST.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Common/typeid_cast.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/ConstraintsDescription.h>
#include <Storages/IStorage.h>
namespace DB
{
@ -34,7 +39,6 @@ public:
{
size_t subquery_depth;
bool is_remote;
size_t external_table_id;
TemporaryTablesMapping & external_tables;
SubqueriesForSets & subqueries_for_sets;
bool & has_global_subqueries;
@ -49,7 +53,6 @@ public:
: WithContext(context_)
, subquery_depth(subquery_depth_)
, is_remote(is_remote_)
, external_table_id(1)
, external_tables(tables)
, subqueries_for_sets(subqueries_for_sets_)
, has_global_subqueries(has_global_subqueries_)
@ -92,48 +95,33 @@ public:
{
/// If this is already an external table, you do not need to add anything. Just remember its presence.
auto temporary_table_name = getIdentifierName(subquery_or_table_name);
bool exists_in_local_map = external_tables.end() != external_tables.find(temporary_table_name);
bool exists_in_local_map = external_tables.contains(temporary_table_name);
bool exists_in_context = static_cast<bool>(getContext()->tryResolveStorageID(
StorageID("", temporary_table_name), Context::ResolveExternal));
if (exists_in_local_map || exists_in_context)
return;
}
String external_table_name = subquery_or_table_name->tryGetAlias();
if (external_table_name.empty())
String alias = subquery_or_table_name->tryGetAlias();
String external_table_name;
if (alias.empty())
{
/// Generate the name for the external table.
external_table_name = "_data" + toString(external_table_id);
while (external_tables.count(external_table_name))
{
++external_table_id;
external_table_name = "_data" + toString(external_table_id);
}
auto hash = subquery_or_table_name->getTreeHash();
external_table_name = fmt::format("_data_{}_{}", hash.first, hash.second);
}
auto interpreter = interpretSubquery(subquery_or_table_name, getContext(), subquery_depth, {});
Block sample = interpreter->getSampleBlock();
NamesAndTypesList columns = sample.getNamesAndTypesList();
auto external_storage_holder = std::make_shared<TemporaryTableHolder>(
getContext(),
ColumnsDescription{columns},
ConstraintsDescription{},
nullptr,
/*create_for_global_subquery*/ true);
StoragePtr external_storage = external_storage_holder->getTable();
else
external_table_name = alias;
/** We replace the subquery with the name of the temporary table.
* It is in this form, the request will go to the remote server.
* This temporary table will go to the remote server, and on its side,
* instead of doing a subquery, you just need to read it.
* TODO We can do better than using alias to name external tables
*/
auto database_and_table_name = std::make_shared<ASTTableIdentifier>(external_table_name);
if (set_alias)
{
String alias = subquery_or_table_name->tryGetAlias();
if (auto * table_name = subquery_or_table_name->as<ASTTableIdentifier>())
if (alias.empty())
alias = table_name->shortName();
@ -151,8 +139,27 @@ public:
else
ast = database_and_table_name;
external_tables[external_table_name] = external_storage_holder;
if (external_tables.contains(external_table_name))
return;
auto interpreter = interpretSubquery(subquery_or_table_name, getContext(), subquery_depth, {});
Block sample = interpreter->getSampleBlock();
NamesAndTypesList columns = sample.getNamesAndTypesList();
auto external_storage_holder = std::make_shared<TemporaryTableHolder>(
getContext(),
ColumnsDescription{columns},
ConstraintsDescription{},
nullptr,
/*create_for_global_subquery*/ true);
StoragePtr external_storage = external_storage_holder->getTable();
external_tables.emplace(external_table_name, external_storage_holder);
/// We need to materialize external tables immediately because reading from distributed
/// tables might generate local plans which can refer to external tables during index
/// analysis. It's too late to populate the external table via CreatingSetsTransform.
if (getContext()->getSettingsRef().use_index_for_in_with_subqueries)
{
auto external_table = external_storage_holder->getTable();

View File

@ -1242,10 +1242,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
}
preliminary_sort();
// If there is no global subqueries, we can run subqueries only when receive them on server.
if (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())
executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets);
}
if (expressions.second_stage || from_aggregation_stage)
@ -1428,7 +1424,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
}
}
if (!subqueries_for_sets.empty() && (expressions.hasHaving() || query_analyzer->hasGlobalSubqueries()))
if (!subqueries_for_sets.empty())
executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets);
}
@ -1892,7 +1888,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
&& limit_length <= std::numeric_limits<UInt64>::max() - limit_offset
&& limit_length + limit_offset < max_block_size)
{
max_block_size = std::max(UInt64(1), limit_length + limit_offset);
max_block_size = std::max(UInt64{1}, limit_length + limit_offset);
max_threads_execute_query = max_streams = 1;
}
@ -2578,11 +2574,11 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan)
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, SubqueriesForSets & subqueries_for_sets)
{
const auto & input_order_info = query_info.input_order_info
? query_info.input_order_info
: (query_info.projection ? query_info.projection->input_order_info : nullptr);
if (input_order_info)
executeMergeSorted(query_plan, input_order_info->order_key_prefix_descr, 0, "before creating sets for subqueries and joins");
// const auto & input_order_info = query_info.input_order_info
// ? query_info.input_order_info
// : (query_info.projection ? query_info.projection->input_order_info : nullptr);
// if (input_order_info)
// executeMergeSorted(query_plan, input_order_info->order_key_prefix_descr, 0, "before creating sets for subqueries and joins");
const Settings & settings = context->getSettingsRef();

View File

@ -0,0 +1,39 @@
#include "threadPoolCallbackRunner.h"
#include <base/scope_guard_safe.h>
#include <Common/CurrentThread.h>
namespace DB
{
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool)
{
return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback)
{
pool->scheduleOrThrow([callback = std::move(callback), thread_group]()
{
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT_SAFE({
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
/// After we detached from the thread_group, parent for memory_tracker inside ThreadStatus will be reset to it's parent.
/// Typically, it may be changes from Process to User.
/// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed.
/// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well.
/// When, finally, we destroy the thread (and the ThreadStatus),
/// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,\
/// and by this time user-level memory tracker may be already destroyed.
///
/// As a work-around, reset memory tracker to total, which is always alive.
CurrentThread::get().memory_tracker.setParent(&total_memory_tracker);
});
callback();
});
};
}
}

View File

@ -0,0 +1,15 @@
#pragma once
#include <Common/ThreadPool.h>
namespace DB
{
/// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously
using CallbackRunner = std::function<void(std::function<void()>)>;
/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool);
}

View File

@ -399,6 +399,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) c
return true;
/// disallow GLOBAL IN, GLOBAL NOT IN
/// TODO why?
if ("globalIn" == function_ptr->name
|| "globalNotIn" == function_ptr->name)
return true;

View File

@ -12,6 +12,7 @@
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTInsertQuery.h>
@ -20,6 +21,7 @@
#include <Storages/StorageFactory.h>
#include <Storages/StorageS3.h>
#include <Storages/StorageS3Settings.h>
#include <Storages/StorageSnapshot.h>
#include <Storages/PartitionedSink.h>
#include <IO/ReadBufferFromS3.h>
@ -374,6 +376,16 @@ static bool checkIfObjectExists(const std::shared_ptr<Aws::S3::S3Client> & clien
return false;
}
// TODO: common thread pool for IO must be used instead after PR #35150
static ThreadPool & getThreadPoolStorageS3()
{
constexpr size_t pool_size = 100;
constexpr size_t queue_size = 1000000;
static ThreadPool pool(pool_size, pool_size, queue_size);
return pool;
}
class StorageS3Sink : public SinkToStorage
{
public:
@ -398,7 +410,7 @@ public:
std::make_unique<WriteBufferFromS3>(
client, bucket, key, min_upload_part_size,
upload_part_size_multiply_factor, upload_part_size_multiply_parts_count_threshold,
max_single_part_upload_size), compression_method, 3);
max_single_part_upload_size, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(getThreadPoolStorageS3())), compression_method, 3);
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings);
}

View File

@ -373,6 +373,11 @@ class SettingsRandomizer:
"priority": lambda: int(abs(random.gauss(0, 2))),
"output_format_parallel_formatting": lambda: random.randint(0, 1),
"input_format_parallel_parsing": lambda: random.randint(0, 1),
"min_chunk_bytes_for_parallel_parsing": lambda: max(1024, int(random.gauss(10 * 1024 * 1024, 5 * 1000 * 1000))),
"max_read_buffer_size": lambda: random.randint(1, 20) if random.random() < 0.1 else random.randint(500000, 1048576),
"prefer_localhost_replica": lambda: random.randint(0, 1),
"max_block_size": lambda: random.randint(8000, 100000),
"max_threads": lambda: random.randint(1, 64),
}
@staticmethod

View File

@ -1,5 +1,7 @@
<clickhouse>
<zookeeper>
<!--<zookeeper_load_balancing>random / in_order / nearest_hostname / first_or_random / round_robin</zookeeper_load_balancing>-->
<zookeeper_load_balancing>random</zookeeper_load_balancing>
<node index="1">
<host>localhost</host>
<port>9181</port>

View File

@ -1459,7 +1459,7 @@
"xor"
"xxHash32"
"xxHash64"
"yandexConsistentHash"
"kostikConsistentHash"
"YEAR"
"yearweek"
"yesterday"

View File

@ -26,7 +26,7 @@
"toUnixTimestamp64Nano"
"toUnixTimestamp64Micro"
"jumpConsistentHash"
"yandexConsistentHash"
"kostikConsistentHash"
"addressToSymbol"
"toJSONString"
"JSON_VALUE"

View File

@ -16,21 +16,28 @@ import traceback
import urllib.parse
import shlex
import urllib3
from cassandra.policies import RoundRobinPolicy
import cassandra.cluster
import psycopg2
import pymongo
import pymysql
import requests
from confluent_kafka.avro.cached_schema_registry_client import (
CachedSchemaRegistryClient,
)
try:
# Please, add modules that required for specific tests only here.
# So contributors will be able to run most tests locally
# without installing tons of unneeded packages that may be not so easy to install.
from cassandra.policies import RoundRobinPolicy
import cassandra.cluster
import psycopg2
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
import pymongo
import pymysql
from confluent_kafka.avro.cached_schema_registry_client import (
CachedSchemaRegistryClient,
)
except Exception as e:
logging.warning(f"Cannot import some modules, some tests may not work: {e}")
from dict2xml import dict2xml
from kazoo.client import KazooClient
from kazoo.exceptions import KazooException
from minio import Minio
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry
from helpers import pytest_xdist_logging_to_separate_files

View File

@ -67,10 +67,10 @@ def started_cluster():
insert into data (key) select * from numbers(10);
create table if not exists dist_one as data engine=Distributed(one_shard, currentDatabase(), data, key);
create table if not exists dist_one_over_dist as data engine=Distributed(one_shard, currentDatabase(), dist_one, yandexConsistentHash(key, 2));
create table if not exists dist_one_over_dist as data engine=Distributed(one_shard, currentDatabase(), dist_one, kostikConsistentHash(key, 2));
create table if not exists dist_two as data engine=Distributed(two_shards, currentDatabase(), data, key);
create table if not exists dist_two_over_dist as data engine=Distributed(two_shards, currentDatabase(), dist_two, yandexConsistentHash(key, 2));
create table if not exists dist_two_over_dist as data engine=Distributed(two_shards, currentDatabase(), dist_two, kostikConsistentHash(key, 2));
"""
)
yield cluster

View File

@ -0,0 +1,19 @@
<clickhouse>
<zookeeper>
<!--<zookeeper_load_balancing> random / in_order / nearest_hostname / first_or_random / round_robin </zookeeper_load_balancing>-->
<zookeeper_load_balancing>random</zookeeper_load_balancing>
<node index="1">
<host>zoo1</host>
<port>2181</port>
</node>
<node index="2">
<host>zoo2</host>
<port>2181</port>
</node>
<node index="3">
<host>zoo3</host>
<port>2181</port>
</node>
<session_timeout_ms>3000</session_timeout_ms>
</zookeeper>
</clickhouse>

View File

@ -0,0 +1,427 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
cluster = ClickHouseCluster(
__file__, zookeeper_config_path="configs/zookeeper_load_balancing.xml"
)
# use 3-letter hostnames, so getHostNameDifference("nod1", "zoo1") will work as expected
node1 = cluster.add_instance(
"nod1", with_zookeeper=True, main_configs=["configs/zookeeper_load_balancing.xml"]
)
node2 = cluster.add_instance(
"nod2", with_zookeeper=True, main_configs=["configs/zookeeper_load_balancing.xml"]
)
node3 = cluster.add_instance(
"nod3", with_zookeeper=True, main_configs=["configs/zookeeper_load_balancing.xml"]
)
def change_balancing(old, new, reload=True):
line = "<zookeeper_load_balancing>{}<"
old_line = line.format(old)
new_line = line.format(new)
for node in [node1, node2, node3]:
node.replace_in_config(
"/etc/clickhouse-server/config.d/zookeeper_load_balancing.xml",
old_line,
new_line,
)
if reload:
node.query("select '{}', '{}'".format(old, new))
node.query("system reload config")
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_first_or_random(started_cluster):
try:
change_balancing("random", "first_or_random")
print(
str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
finally:
change_balancing("first_or_random", "random", reload=False)
def test_in_order(started_cluster):
try:
change_balancing("random", "in_order")
print(
str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
finally:
change_balancing("in_order", "random", reload=False)
def test_nearest_hostname(started_cluster):
try:
change_balancing("random", "nearest_hostname")
print(
str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo3_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
finally:
change_balancing("nearest_hostname", "random", reload=False)
def test_round_robin(started_cluster):
pm = PartitionManager()
try:
pm._add_rule(
{
"source": node1.ip_address,
"destination": cluster.get_instance_ip("zoo1"),
"action": "REJECT --reject-with tcp-reset",
}
)
pm._add_rule(
{
"source": node2.ip_address,
"destination": cluster.get_instance_ip("zoo1"),
"action": "REJECT --reject-with tcp-reset",
}
)
pm._add_rule(
{
"source": node3.ip_address,
"destination": cluster.get_instance_ip("zoo1"),
"action": "REJECT --reject-with tcp-reset",
}
)
change_balancing("random", "round_robin")
print(
str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node1.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node2.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
print(
str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED",
],
privileged=True,
user="root",
)
)
)
assert (
"1"
== str(
node3.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
)
finally:
pm.heal_all()
change_balancing("round_robin", "random", reload=False)

View File

@ -3,7 +3,7 @@
<substitution>
<name>hash_func</name>
<values>
<value>yandexConsistentHash</value>
<value>kostikConsistentHash</value>
<value>jumpConsistentHash</value>
</values>
</substitution>

View File

@ -99,6 +99,6 @@ abc
1
1
Остальные
Яндекс
Bigmir)net
Google
Остальные

View File

@ -8,10 +8,10 @@ SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], 0) FROM sys
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], -1) FROM system.numbers LIMIT 10;
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], -1.1) FROM system.numbers LIMIT 10;
SELECT transform(toString(number), ['3', '5', '7'], [111, 222.2, 333], 1) FROM system.numbers LIMIT 10;
SELECT transform(1, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
SELECT transform(2, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
SELECT transform(3, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
SELECT transform(4, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
SELECT transform(1, [2, 3], ['Bigmir)net', 'Google'], 'Остальные') AS title;
SELECT transform(2, [2, 3], ['Bigmir)net', 'Google'], 'Остальные') AS title;
SELECT transform(3, [2, 3], ['Bigmir)net', 'Google'], 'Остальные') AS title;
SELECT transform(4, [2, 3], ['Bigmir)net', 'Google'], 'Остальные') AS title;
SELECT transform('hello', 'wrong', 1); -- { serverError 43 }
SELECT transform('hello', ['wrong'], 1); -- { serverError 43 }
SELECT transform('hello', ['wrong'], [1]); -- { serverError 43 }

View File

@ -79,6 +79,6 @@ abc
1
1
Остальные
Яндекс
Meta.ua
Google
Остальные

View File

@ -6,7 +6,7 @@ SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], materialize
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], materialize(-1)) FROM system.numbers LIMIT 10;
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], materialize(-1.1)) FROM system.numbers LIMIT 10;
SELECT transform(toString(number), ['3', '5', '7'], [111, 222.2, 333], materialize(1)) FROM system.numbers LIMIT 10;
SELECT transform(1, [2, 3], ['Яндекс', 'Google'], materialize('Остальные')) AS title;
SELECT transform(2, [2, 3], ['Яндекс', 'Google'], materialize('Остальные')) AS title;
SELECT transform(3, [2, 3], ['Яндекс', 'Google'], materialize('Остальные')) AS title;
SELECT transform(4, [2, 3], ['Яндекс', 'Google'], materialize('Остальные')) AS title;
SELECT transform(1, [2, 3], ['Meta.ua', 'Google'], materialize('Остальные')) AS title;
SELECT transform(2, [2, 3], ['Meta.ua', 'Google'], materialize('Остальные')) AS title;
SELECT transform(3, [2, 3], ['Meta.ua', 'Google'], materialize('Остальные')) AS title;
SELECT transform(4, [2, 3], ['Meta.ua', 'Google'], materialize('Остальные')) AS title;

View File

@ -34,25 +34,25 @@ Hello, World
0,1,2,3,4,5,6,7
0,1,2,3,4,5,6,7,8
yandex
yandex google
yandex google test
yandex google test 123
yandex google test 123
yandex google test 123 hello
yandex google test 123 hello world
yandex google test 123 hello world goodbye
yandex google test 123 hello world goodbye xyz
yandex google test 123 hello world goodbye xyz yandex
yandex google test 123 hello world goodbye xyz yandex google
yandex google test 123 hello world goodbye xyz yandex google test
yandex google test 123 hello world goodbye xyz yandex google test 123
yandex google test 123 hello world goodbye xyz yandex google test 123
yandex google test 123 hello world goodbye xyz yandex google test 123 hello
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world goodbye
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world goodbye xyz
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world goodbye xyz yandex
meta.ua
meta.ua google
meta.ua google test
meta.ua google test 123
meta.ua google test 123
meta.ua google test 123 hello
meta.ua google test 123 hello world
meta.ua google test 123 hello world goodbye
meta.ua google test 123 hello world goodbye xyz
meta.ua google test 123 hello world goodbye xyz meta.ua
meta.ua google test 123 hello world goodbye xyz meta.ua google
meta.ua google test 123 hello world goodbye xyz meta.ua google test
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123 hello
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123 hello world
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123 hello world goodbye
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123 hello world goodbye xyz
meta.ua google test 123 hello world goodbye xyz meta.ua google test 123 hello world goodbye xyz meta.ua
0
01

View File

@ -6,7 +6,7 @@ SELECT arrayStringConcat(emptyArrayString());
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number)), '') FROM system.numbers LIMIT 10;
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number)), ',') FROM system.numbers LIMIT 10;
SELECT arrayStringConcat(arrayMap(x -> transform(x, [0, 1, 2, 3, 4, 5, 6, 7, 8], ['yandex', 'google', 'test', '123', '', 'hello', 'world', 'goodbye', 'xyz'], ''), arrayMap(x -> x % 9, range(number))), ' ') FROM system.numbers LIMIT 20;
SELECT arrayStringConcat(arrayMap(x -> transform(x, [0, 1, 2, 3, 4, 5, 6, 7, 8], ['meta.ua', 'google', 'test', '123', '', 'hello', 'world', 'goodbye', 'xyz'], ''), arrayMap(x -> x % 9, range(number))), ' ') FROM system.numbers LIMIT 20;
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number % 4))) FROM system.numbers LIMIT 10;
SELECT arrayStringConcat([Null, 'hello', Null, 'world', Null, 'xyz', 'def', Null], ';');
SELECT arrayStringConcat([Null::Nullable(String), Null::Nullable(String)], ';');

View File

@ -1,8 +1,8 @@
['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h']
['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g'] ['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g']
b d f d f h b d d h f h b d f d f h b d d h f h
http://yandex.ru/?c=d http://yandex.ru/?a=b http://yandex.ru/?a=b&c=d# http://yandex.ru/?a&c=d#e=f http://yandex.ru/?a#e=f http://yandex.ru/?a&c=d# http://yandex.ru/?a=b&c=d#e=f http://yandex.ru/?c=d#e http://yandex.ru/?a=b#e http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b#e&g=h http://yandex.ru/?a=b&c=d#e&g=h http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b&c=d#test?e=f&g=h http://yandex.ru/?a=b&c=d#test?g=h http://yandex.ru/?a=b&c=d#test?e=f //yandex.ru/?c=d //yandex.ru/?a=b //yandex.ru/?a=b&c=d# //yandex.ru/?a&c=d#e=f //yandex.ru/?a#e=f //yandex.ru/?a&c=d# //yandex.ru/?a=b&c=d#e=f //yandex.ru/?c=d#e //yandex.ru/?a=b#e //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b#e&g=h //yandex.ru/?a=b&c=d#e&g=h //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b&c=d#test?e=f&g=h //yandex.ru/?a=b&c=d#test?g=h //yandex.ru/?a=b&c=d#test?e=f
http://bigmir.net/?c=d http://bigmir.net/?a=b http://bigmir.net/?a=b&c=d# http://bigmir.net/?a&c=d#e=f http://bigmir.net/?a#e=f http://bigmir.net/?a&c=d# http://bigmir.net/?a=b&c=d#e=f http://bigmir.net/?c=d#e http://bigmir.net/?a=b#e http://bigmir.net/?a=b&c=d#e http://bigmir.net/?a=b#e&g=h http://bigmir.net/?a=b&c=d#e&g=h http://bigmir.net/?a=b&c=d#e http://bigmir.net/?a=b&c=d#test?e=f&g=h http://bigmir.net/?a=b&c=d#test?g=h http://bigmir.net/?a=b&c=d#test?e=f //bigmir.net/?c=d //bigmir.net/?a=b //bigmir.net/?a=b&c=d# //bigmir.net/?a&c=d#e=f //bigmir.net/?a#e=f //bigmir.net/?a&c=d# //bigmir.net/?a=b&c=d#e=f //bigmir.net/?c=d#e //bigmir.net/?a=b#e //bigmir.net/?a=b&c=d#e //bigmir.net/?a=b#e&g=h //bigmir.net/?a=b&c=d#e&g=h //bigmir.net/?a=b&c=d#e //bigmir.net/?a=b&c=d#test?e=f&g=h //bigmir.net/?a=b&c=d#test?g=h //bigmir.net/?a=b&c=d#test?e=f
['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h']
['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g'] ['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g']
b d f d f h b d d h f h b d f d f h b d d h f h
http://yandex.ru/?c=d http://yandex.ru/?a=b http://yandex.ru/?a=b&c=d# http://yandex.ru/?a&c=d#e=f http://yandex.ru/?a#e=f http://yandex.ru/?a&c=d# http://yandex.ru/?a=b&c=d#e=f http://yandex.ru/?c=d#e http://yandex.ru/?a=b#e http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b#e&g=h http://yandex.ru/?a=b&c=d#e&g=h http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b&c=d#test?e=f&g=h http://yandex.ru/?a=b&c=d#test?g=h http://yandex.ru/?a=b&c=d#test?e=f //yandex.ru/?c=d //yandex.ru/?a=b //yandex.ru/?a=b&c=d# //yandex.ru/?a&c=d#e=f //yandex.ru/?a#e=f //yandex.ru/?a&c=d# //yandex.ru/?a=b&c=d#e=f //yandex.ru/?c=d#e //yandex.ru/?a=b#e //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b#e&g=h //yandex.ru/?a=b&c=d#e&g=h //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b&c=d#test?e=f&g=h //yandex.ru/?a=b&c=d#test?g=h //yandex.ru/?a=b&c=d#test?e=f
http://bigmir.net/?c=d http://bigmir.net/?a=b http://bigmir.net/?a=b&c=d# http://bigmir.net/?a&c=d#e=f http://bigmir.net/?a#e=f http://bigmir.net/?a&c=d# http://bigmir.net/?a=b&c=d#e=f http://bigmir.net/?c=d#e http://bigmir.net/?a=b#e http://bigmir.net/?a=b&c=d#e http://bigmir.net/?a=b#e&g=h http://bigmir.net/?a=b&c=d#e&g=h http://bigmir.net/?a=b&c=d#e http://bigmir.net/?a=b&c=d#test?e=f&g=h http://bigmir.net/?a=b&c=d#test?g=h http://bigmir.net/?a=b&c=d#test?e=f //bigmir.net/?c=d //bigmir.net/?a=b //bigmir.net/?a=b&c=d# //bigmir.net/?a&c=d#e=f //bigmir.net/?a#e=f //bigmir.net/?a&c=d# //bigmir.net/?a=b&c=d#e=f //bigmir.net/?c=d#e //bigmir.net/?a=b#e //bigmir.net/?a=b&c=d#e //bigmir.net/?a=b#e&g=h //bigmir.net/?a=b&c=d#e&g=h //bigmir.net/?a=b&c=d#e //bigmir.net/?a=b&c=d#test?e=f&g=h //bigmir.net/?a=b&c=d#test?g=h //bigmir.net/?a=b&c=d#test?e=f

View File

@ -1,200 +1,200 @@
SELECT
extractURLParameters('http://yandex.ru/?a=b&c=d'),
extractURLParameters('http://yandex.ru/?a=b&c=d#e=f'),
extractURLParameters('http://yandex.ru/?a&c=d#e=f'),
extractURLParameters('http://yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameters('http://yandex.ru/?a=b&c=d#e'),
extractURLParameters('http://yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameters('http://yandex.ru/?a=b&c=d#test?e=f&g=h'),
extractURLParameters('//yandex.ru/?a=b&c=d'),
extractURLParameters('//yandex.ru/?a=b&c=d#e=f'),
extractURLParameters('//yandex.ru/?a&c=d#e=f'),
extractURLParameters('//yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameters('//yandex.ru/?a=b&c=d#e'),
extractURLParameters('//yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameters('//yandex.ru/?a=b&c=d#test?e=f&g=h');
extractURLParameters('http://bigmir.net/?a=b&c=d'),
extractURLParameters('http://bigmir.net/?a=b&c=d#e=f'),
extractURLParameters('http://bigmir.net/?a&c=d#e=f'),
extractURLParameters('http://bigmir.net/?a=b&c=d#e=f&g=h'),
extractURLParameters('http://bigmir.net/?a=b&c=d#e'),
extractURLParameters('http://bigmir.net/?a=b&c=d#e&g=h'),
extractURLParameters('http://bigmir.net/?a=b&c=d#test?e=f&g=h'),
extractURLParameters('//bigmir.net/?a=b&c=d'),
extractURLParameters('//bigmir.net/?a=b&c=d#e=f'),
extractURLParameters('//bigmir.net/?a&c=d#e=f'),
extractURLParameters('//bigmir.net/?a=b&c=d#e=f&g=h'),
extractURLParameters('//bigmir.net/?a=b&c=d#e'),
extractURLParameters('//bigmir.net/?a=b&c=d#e&g=h'),
extractURLParameters('//bigmir.net/?a=b&c=d#test?e=f&g=h');
SELECT
extractURLParameterNames('http://yandex.ru/?a=b&c=d'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e=f'),
extractURLParameterNames('http://yandex.ru/?a&c=d#e=f'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#test?e=f&g=h'),
extractURLParameterNames('//yandex.ru/?a=b&c=d'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e=f'),
extractURLParameterNames('//yandex.ru/?a&c=d#e=f'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#test?e=f&g=h');
extractURLParameterNames('http://bigmir.net/?a=b&c=d'),
extractURLParameterNames('http://bigmir.net/?a=b&c=d#e=f'),
extractURLParameterNames('http://bigmir.net/?a&c=d#e=f'),
extractURLParameterNames('http://bigmir.net/?a=b&c=d#e=f&g=h'),
extractURLParameterNames('http://bigmir.net/?a=b&c=d#e'),
extractURLParameterNames('http://bigmir.net/?a=b&c=d#e&g=h'),
extractURLParameterNames('http://bigmir.net/?a=b&c=d#test?e=f&g=h'),
extractURLParameterNames('//bigmir.net/?a=b&c=d'),
extractURLParameterNames('//bigmir.net/?a=b&c=d#e=f'),
extractURLParameterNames('//bigmir.net/?a&c=d#e=f'),
extractURLParameterNames('//bigmir.net/?a=b&c=d#e=f&g=h'),
extractURLParameterNames('//bigmir.net/?a=b&c=d#e'),
extractURLParameterNames('//bigmir.net/?a=b&c=d#e&g=h'),
extractURLParameterNames('//bigmir.net/?a=b&c=d#test?e=f&g=h');
SELECT
extractURLParameter('http://yandex.ru/?a=b&c=d', 'a'),
extractURLParameter('http://yandex.ru/?a=b&c=d', 'c'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e=f', 'e'),
extractURLParameter('http://yandex.ru/?a&c=d#e=f', 'a'),
extractURLParameter('http://yandex.ru/?a&c=d#e=f', 'c'),
extractURLParameter('http://yandex.ru/?a&c=d#e=f', 'e'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e=f&g=h', 'g'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e', 'a'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e', 'c'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e', 'e'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'c'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'e'),
extractURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'g'),
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'g'),
extractURLParameter('//yandex.ru/?a=b&c=d', 'a'),
extractURLParameter('//yandex.ru/?a=b&c=d', 'c'),
extractURLParameter('//yandex.ru/?a=b&c=d#e=f', 'e'),
extractURLParameter('//yandex.ru/?a&c=d#e=f', 'a'),
extractURLParameter('//yandex.ru/?a&c=d#e=f', 'c'),
extractURLParameter('//yandex.ru/?a&c=d#e=f', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#e=f&g=h', 'g'),
extractURLParameter('//yandex.ru/?a=b&c=d#e', 'a'),
extractURLParameter('//yandex.ru/?a=b&c=d#e', 'c'),
extractURLParameter('//yandex.ru/?a=b&c=d#e', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'c'),
extractURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'g'),
extractURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
extractURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'g');
extractURLParameter('http://bigmir.net/?a=b&c=d', 'a'),
extractURLParameter('http://bigmir.net/?a=b&c=d', 'c'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e=f', 'e'),
extractURLParameter('http://bigmir.net/?a&c=d#e=f', 'a'),
extractURLParameter('http://bigmir.net/?a&c=d#e=f', 'c'),
extractURLParameter('http://bigmir.net/?a&c=d#e=f', 'e'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e=f&g=h', 'g'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e', 'a'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e', 'c'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e', 'e'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', 'c'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', 'e'),
extractURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', 'g'),
extractURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', 'test'),
extractURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', 'e'),
extractURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', 'g'),
extractURLParameter('//bigmir.net/?a=b&c=d', 'a'),
extractURLParameter('//bigmir.net/?a=b&c=d', 'c'),
extractURLParameter('//bigmir.net/?a=b&c=d#e=f', 'e'),
extractURLParameter('//bigmir.net/?a&c=d#e=f', 'a'),
extractURLParameter('//bigmir.net/?a&c=d#e=f', 'c'),
extractURLParameter('//bigmir.net/?a&c=d#e=f', 'e'),
extractURLParameter('//bigmir.net/?a=b&c=d#e=f&g=h', 'g'),
extractURLParameter('//bigmir.net/?a=b&c=d#e', 'a'),
extractURLParameter('//bigmir.net/?a=b&c=d#e', 'c'),
extractURLParameter('//bigmir.net/?a=b&c=d#e', 'e'),
extractURLParameter('//bigmir.net/?a=b&c=d#e&g=h', 'c'),
extractURLParameter('//bigmir.net/?a=b&c=d#e&g=h', 'e'),
extractURLParameter('//bigmir.net/?a=b&c=d#e&g=h', 'g'),
extractURLParameter('//bigmir.net/?a=b&c=d#test?e=f&g=h', 'test'),
extractURLParameter('//bigmir.net/?a=b&c=d#test?e=f&g=h', 'e'),
extractURLParameter('//bigmir.net/?a=b&c=d#test?e=f&g=h', 'g');
SELECT
cutURLParameter('http://yandex.ru/?a=b&c=d', 'a'),
cutURLParameter('http://yandex.ru/?a=b&c=d', 'c'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e=f', 'e'),
cutURLParameter('http://yandex.ru/?a&c=d#e=f', 'a'),
cutURLParameter('http://yandex.ru/?a&c=d#e=f', 'c'),
cutURLParameter('http://yandex.ru/?a&c=d#e=f', 'e'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e=f&g=h', 'g'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e', 'a'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e', 'c'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e', 'e'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'c'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'e'),
cutURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'g'),
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'g'),
cutURLParameter('//yandex.ru/?a=b&c=d', 'a'),
cutURLParameter('//yandex.ru/?a=b&c=d', 'c'),
cutURLParameter('//yandex.ru/?a=b&c=d#e=f', 'e'),
cutURLParameter('//yandex.ru/?a&c=d#e=f', 'a'),
cutURLParameter('//yandex.ru/?a&c=d#e=f', 'c'),
cutURLParameter('//yandex.ru/?a&c=d#e=f', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#e=f&g=h', 'g'),
cutURLParameter('//yandex.ru/?a=b&c=d#e', 'a'),
cutURLParameter('//yandex.ru/?a=b&c=d#e', 'c'),
cutURLParameter('//yandex.ru/?a=b&c=d#e', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'c'),
cutURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'g'),
cutURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
cutURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'g');
cutURLParameter('http://bigmir.net/?a=b&c=d', 'a'),
cutURLParameter('http://bigmir.net/?a=b&c=d', 'c'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e=f', 'e'),
cutURLParameter('http://bigmir.net/?a&c=d#e=f', 'a'),
cutURLParameter('http://bigmir.net/?a&c=d#e=f', 'c'),
cutURLParameter('http://bigmir.net/?a&c=d#e=f', 'e'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e=f&g=h', 'g'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e', 'a'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e', 'c'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e', 'e'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', 'c'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', 'e'),
cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', 'g'),
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', 'test'),
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', 'e'),
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', 'g'),
cutURLParameter('//bigmir.net/?a=b&c=d', 'a'),
cutURLParameter('//bigmir.net/?a=b&c=d', 'c'),
cutURLParameter('//bigmir.net/?a=b&c=d#e=f', 'e'),
cutURLParameter('//bigmir.net/?a&c=d#e=f', 'a'),
cutURLParameter('//bigmir.net/?a&c=d#e=f', 'c'),
cutURLParameter('//bigmir.net/?a&c=d#e=f', 'e'),
cutURLParameter('//bigmir.net/?a=b&c=d#e=f&g=h', 'g'),
cutURLParameter('//bigmir.net/?a=b&c=d#e', 'a'),
cutURLParameter('//bigmir.net/?a=b&c=d#e', 'c'),
cutURLParameter('//bigmir.net/?a=b&c=d#e', 'e'),
cutURLParameter('//bigmir.net/?a=b&c=d#e&g=h', 'c'),
cutURLParameter('//bigmir.net/?a=b&c=d#e&g=h', 'e'),
cutURLParameter('//bigmir.net/?a=b&c=d#e&g=h', 'g'),
cutURLParameter('//bigmir.net/?a=b&c=d#test?e=f&g=h', 'test'),
cutURLParameter('//bigmir.net/?a=b&c=d#test?e=f&g=h', 'e'),
cutURLParameter('//bigmir.net/?a=b&c=d#test?e=f&g=h', 'g');
SELECT
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e=f')),
extractURLParameters(materialize('http://yandex.ru/?a&c=d#e=f')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e=f')),
extractURLParameters(materialize('//yandex.ru/?a&c=d#e=f')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'));
extractURLParameters(materialize('http://bigmir.net/?a=b&c=d')),
extractURLParameters(materialize('http://bigmir.net/?a=b&c=d#e=f')),
extractURLParameters(materialize('http://bigmir.net/?a&c=d#e=f')),
extractURLParameters(materialize('http://bigmir.net/?a=b&c=d#e=f&g=h')),
extractURLParameters(materialize('http://bigmir.net/?a=b&c=d#e')),
extractURLParameters(materialize('http://bigmir.net/?a=b&c=d#e&g=h')),
extractURLParameters(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h')),
extractURLParameters(materialize('//bigmir.net/?a=b&c=d')),
extractURLParameters(materialize('//bigmir.net/?a=b&c=d#e=f')),
extractURLParameters(materialize('//bigmir.net/?a&c=d#e=f')),
extractURLParameters(materialize('//bigmir.net/?a=b&c=d#e=f&g=h')),
extractURLParameters(materialize('//bigmir.net/?a=b&c=d#e')),
extractURLParameters(materialize('//bigmir.net/?a=b&c=d#e&g=h')),
extractURLParameters(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'));
SELECT
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e=f')),
extractURLParameterNames(materialize('http://yandex.ru/?a&c=d#e=f')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e=f')),
extractURLParameterNames(materialize('//yandex.ru/?a&c=d#e=f')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'));
extractURLParameterNames(materialize('http://bigmir.net/?a=b&c=d')),
extractURLParameterNames(materialize('http://bigmir.net/?a=b&c=d#e=f')),
extractURLParameterNames(materialize('http://bigmir.net/?a&c=d#e=f')),
extractURLParameterNames(materialize('http://bigmir.net/?a=b&c=d#e=f&g=h')),
extractURLParameterNames(materialize('http://bigmir.net/?a=b&c=d#e')),
extractURLParameterNames(materialize('http://bigmir.net/?a=b&c=d#e&g=h')),
extractURLParameterNames(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h')),
extractURLParameterNames(materialize('//bigmir.net/?a=b&c=d')),
extractURLParameterNames(materialize('//bigmir.net/?a=b&c=d#e=f')),
extractURLParameterNames(materialize('//bigmir.net/?a&c=d#e=f')),
extractURLParameterNames(materialize('//bigmir.net/?a=b&c=d#e=f&g=h')),
extractURLParameterNames(materialize('//bigmir.net/?a=b&c=d#e')),
extractURLParameterNames(materialize('//bigmir.net/?a=b&c=d#e&g=h')),
extractURLParameterNames(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'));
SELECT
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d'), 'a'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d'), 'c'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e=f'), 'e'),
extractURLParameter(materialize('http://yandex.ru/?a&c=d#e=f'), 'a'),
extractURLParameter(materialize('http://yandex.ru/?a&c=d#e=f'), 'c'),
extractURLParameter(materialize('http://yandex.ru/?a&c=d#e=f'), 'e'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e=f&g=h'), 'g'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e'), 'a'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e'), 'c'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e'), 'e'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'c'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'e'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'g'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'a'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'a'),
extractURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f&g=h'), 'g'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'a'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'g'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g');
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d'), 'a'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d'), 'c'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e=f'), 'e'),
extractURLParameter(materialize('http://bigmir.net/?a&c=d#e=f'), 'a'),
extractURLParameter(materialize('http://bigmir.net/?a&c=d#e=f'), 'c'),
extractURLParameter(materialize('http://bigmir.net/?a&c=d#e=f'), 'e'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e=f&g=h'), 'g'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e'), 'a'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e'), 'c'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e'), 'e'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), 'c'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), 'e'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), 'g'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), 'test'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), 'e'),
extractURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), 'g'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d'), 'a'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d'), 'c'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f'), 'e'),
extractURLParameter(materialize('//bigmir.net/?a&c=d#e=f'), 'a'),
extractURLParameter(materialize('//bigmir.net/?a&c=d#e=f'), 'c'),
extractURLParameter(materialize('//bigmir.net/?a&c=d#e=f'), 'e'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f&g=h'), 'g'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e'), 'a'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e'), 'c'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e'), 'e'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e&g=h'), 'c'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e&g=h'), 'e'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#e&g=h'), 'g'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'), 'test'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'), 'e'),
extractURLParameter(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'), 'g');
SELECT
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d'), 'a'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d'), 'c'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e=f'), 'e'),
cutURLParameter(materialize('http://yandex.ru/?a&c=d#e=f'), 'a'),
cutURLParameter(materialize('http://yandex.ru/?a&c=d#e=f'), 'c'),
cutURLParameter(materialize('http://yandex.ru/?a&c=d#e=f'), 'e'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e=f&g=h'), 'g'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e'), 'a'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e'), 'c'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e'), 'e'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'c'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'e'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'g'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'a'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'a'),
cutURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f&g=h'), 'g'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'a'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'g'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g');
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), 'a'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), 'c'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e=f'), 'e'),
cutURLParameter(materialize('http://bigmir.net/?a&c=d#e=f'), 'a'),
cutURLParameter(materialize('http://bigmir.net/?a&c=d#e=f'), 'c'),
cutURLParameter(materialize('http://bigmir.net/?a&c=d#e=f'), 'e'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e=f&g=h'), 'g'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e'), 'a'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e'), 'c'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e'), 'e'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), 'c'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), 'e'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), 'g'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), 'test'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), 'e'),
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), 'g'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), 'a'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), 'c'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f'), 'e'),
cutURLParameter(materialize('//bigmir.net/?a&c=d#e=f'), 'a'),
cutURLParameter(materialize('//bigmir.net/?a&c=d#e=f'), 'c'),
cutURLParameter(materialize('//bigmir.net/?a&c=d#e=f'), 'e'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f&g=h'), 'g'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e'), 'a'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e'), 'c'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e'), 'e'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e&g=h'), 'c'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e&g=h'), 'e'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e&g=h'), 'g'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'), 'test'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'), 'e'),
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#test?e=f&g=h'), 'g');

View File

@ -1,3 +1,3 @@
canada congo net-domena
yandex yandex yandex яндекс yandex
meta bigmir yahoo гугл meta
canada hello hello canada

View File

@ -4,12 +4,12 @@ SELECT
firstSignificantSubdomain('http://pochemu.net-domena.ru') AS why;
SELECT
firstSignificantSubdomain('ftp://www.yandex.com.tr/news.html'),
firstSignificantSubdomain('https://www.yandex.ua/news.html'),
firstSignificantSubdomain('magnet:yandex.abc'),
firstSignificantSubdomain('ftp://www.yandex.co.uk/news.html'),
firstSignificantSubdomain('https://api.www3.static.dev.ввв.яндекс.рф'),
firstSignificantSubdomain('//www.yandex.com.tr/news.html');
firstSignificantSubdomain('ftp://www.meta.com.ua/news.html'),
firstSignificantSubdomain('https://www.bigmir.net/news.html'),
firstSignificantSubdomain('magnet:ukr.abc'),
firstSignificantSubdomain('ftp://www.yahoo.co.jp/news.html'),
firstSignificantSubdomain('https://api.www3.static.dev.ввв.гугл.ком'),
firstSignificantSubdomain('//www.meta.com.ua/news.html');
SELECT
firstSignificantSubdomain('http://hello.canada.c'),

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tags: no-fasttest, no-random-settings
# set -x

View File

@ -1,6 +1,6 @@
-- Tags: no-fasttest
SELECT jumpConsistentHash(1, 1), jumpConsistentHash(42, 57), jumpConsistentHash(256, 1024), jumpConsistentHash(3735883980, 1), jumpConsistentHash(3735883980, 666), jumpConsistentHash(16045690984833335023, 255);
SELECT yandexConsistentHash(16045690984833335023, 1), yandexConsistentHash(16045690984833335023, 2), yandexConsistentHash(16045690984833335023, 3), yandexConsistentHash(16045690984833335023, 4), yandexConsistentHash(16045690984833335023, 173), yandexConsistentHash(16045690984833335023, 255);
SELECT kostikConsistentHash(16045690984833335023, 1), kostikConsistentHash(16045690984833335023, 2), kostikConsistentHash(16045690984833335023, 3), kostikConsistentHash(16045690984833335023, 4), kostikConsistentHash(16045690984833335023, 173), kostikConsistentHash(16045690984833335023, 255);
SELECT jumpConsistentHash(intHash64(number), 787) FROM system.numbers LIMIT 1000000, 2;
SELECT yandexConsistentHash(16045690984833335023+number-number, 120) FROM system.numbers LIMIT 1000000, 2;
SELECT kostikConsistentHash(16045690984833335023+number-number, 120) FROM system.numbers LIMIT 1000000, 2;

View File

@ -1,23 +1,23 @@
{"total":"1","domain":"baidu.com"}
{"total":"2","domain":"facebook.com"}
{"total":"1","domain":"google.com"}
{"total":"2","domain":"yandex.ru"}
{"total":"2","domain":"meta.ua"}
{"total":"1","domain":"baidu.com"}
{"total":"2","domain":"facebook.com"}
{"total":"1","domain":"google.com"}
{"total":"2","domain":"yandex.ru"}
{"total":"2","domain":"meta.ua"}
1 baidu.com
2 facebook.com
1 google.com
2 yandex.ru
2 meta.ua
1 baidu.com
2 facebook.com
1 google.com
2 yandex.ru
2 meta.ua
1 baidu.com
1 google.com
2 facebook.com
2 yandex.ru
2 meta.ua
1
1
2
@ -25,4 +25,4 @@
baidu.com
google.com
facebook.com
yandex.ru
meta.ua

View File

@ -4,8 +4,8 @@ DROP TABLE IF EXISTS transactions;
CREATE TABLE clicks (domain String) ENGINE = Memory;
CREATE TABLE transactions (domain String) ENGINE = Memory;
INSERT INTO clicks VALUES ('facebook.com'), ('yandex.ru'), ('google.com');
INSERT INTO transactions VALUES ('facebook.com'), ('yandex.ru'), ('baidu.com');
INSERT INTO clicks VALUES ('facebook.com'), ('meta.ua'), ('google.com');
INSERT INTO transactions VALUES ('facebook.com'), ('meta.ua'), ('baidu.com');
SELECT

View File

@ -25,7 +25,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE string_test_table (val String) ENGINE
${CLICKHOUSE_CLIENT} --query="CREATE TABLE fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'youtube' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;"
${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES string_test_table;"
@ -40,7 +40,7 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO fixed_string_test_table VALUES ('0'),
# 131072 -> 17 bit is 1
${CLICKHOUSE_CLIENT} --query="INSERT INTO signed_integer_test_table VALUES (-2), (0), (2), (2), (131072), (131073), (131073);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO unsigned_integer_test_table VALUES (0), (2), (2), (131072), (131073), (131073);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO enum_test_table VALUES ('hello'), ('world'), ('world'), ('yandex'), ('clickhouse'), ('clickhouse');"
${CLICKHOUSE_CLIENT} --query="INSERT INTO enum_test_table VALUES ('hello'), ('world'), ('world'), ('youtube'), ('clickhouse'), ('clickhouse');"
${CLICKHOUSE_CLIENT} --query="INSERT INTO date_test_table VALUES (1), (2), (2), (256), (257), (257);"
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g')

View File

@ -113,112 +113,112 @@
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
0
привет как дела?... Херсон 600
пап привет как дела - Яндекс.Видео 684
привет как дела клип - Яндекс.Видео 692
привет братан как дела - Яндекс.Видео 707
пап привет как дела - TUT.BY 625
привет как дела клип - TUT.BY 636
привет братан как дела - TUT.BY 657
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
привет 1000
0
http://metric.ru/ 765
http://metris.ru/ 765
http://metrika.ru/ 778
http://autometric.ru/ 810
http://metrica.yandex.com/ 846
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
привет братан как дела - Яндекс.Видео 500
пап привет как дела - TUT.BY 333
привет как дела клип - TUT.BY 350
привет братан как дела - TUT.BY 429
привет 529
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 436
привет как дела?... Херсон 459
пап привет как дела - Яндекс.Видео 511
привет как дела клип - TUT.BY 500
привет братан как дела - TUT.BY 524
привет 529
привет как дела клип - Яндекс.Видео 565
привет братан как дела - Яндекс.Видео 583
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
http://top.bigmir.net/ 920
1000
пап привет как дела - Яндекс.Видео 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://metrica.yandex.com/ 655
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 619
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
http://metrica.yandex.com/ 724
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 714
http://top.bigmir.net/ 667
http://metrika.ru/ 900
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
0
@ -335,135 +335,135 @@ http://metris.ru/ 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
0
привет как дела?... Херсон 600
пап привет как дела - Яндекс.Видео 684
привет как дела клип - Яндекс.Видео 692
привет братан как дела - Яндекс.Видео 707
пап привет как дела - TUT.BY 625
привет как дела клип - TUT.BY 636
привет братан как дела - TUT.BY 657
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
привет 1000
0
http://metric.ru/ 765
http://metris.ru/ 765
http://metrika.ru/ 778
http://autometric.ru/ 810
http://metrica.yandex.com/ 846
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
привет братан как дела - Яндекс.Видео 500
пап привет как дела - TUT.BY 333
привет как дела клип - TUT.BY 350
привет братан как дела - TUT.BY 429
привет 529
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 436
привет как дела?... Херсон 459
пап привет как дела - Яндекс.Видео 511
привет как дела клип - TUT.BY 500
привет братан как дела - TUT.BY 524
привет 529
привет как дела клип - Яндекс.Видео 565
привет братан как дела - Яндекс.Видео 583
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
http://top.bigmir.net/ 920
1000
пап привет как дела - Яндекс.Видео 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
http://top.bigmir.net/ 920
1000
пап привет как дела - Яндекс.Видео 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://metrica.yandex.com/ 655
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 619
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
http://metrica.yandex.com/ 724
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 714
http://top.bigmir.net/ 667
http://metrika.ru/ 900
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
привет как дела клип - TUT.BY 0
пап привет как дела - TUT.BY 208
привет братан как дела - TUT.BY 286
привет как дела?... Херсон 490
привет 742
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 0
пап привет как дела - Яндекс.Видео 169
привет братан как дела - Яндекс.Видео 235
привет как дела?... Херсон 544
привет 784
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
0
0
0
@ -576,82 +576,82 @@ http://metris.ru/ 1000
111
429
1000
пап привет как дела - TUT.BY 242
привет как дела?... Херсон 254
пап привет как дела - Яндекс.Видео 398
привет как дела клип - Яндекс.Видео 412
привет братан как дела - Яндекс.Видео 461
привет как дела клип - TUT.BY 265
привет братан как дела - TUT.BY 333
привет 471
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 303
привет как дела?... Херсон 343
пап привет как дела - Яндекс.Видео 446
привет как дела клип - TUT.BY 353
привет братан как дела - TUT.BY 389
привет 471
привет как дела клип - Яндекс.Видео 482
привет братан как дела - Яндекс.Видео 506
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://metrica.yandex.com/ 704
http://autometric.ru/ 727
http://metris.ru/ 778
http://metrika.ru/ 789
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 684
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://autometric.ru/ 727
http://metrica.yandex.com/ 778
http://metris.ru/ 778
http://metrika.ru/ 789
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 769
http://top.bigmir.net/ 727
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
0
@ -765,91 +765,91 @@ http://metris.ru/ 1000
111
600
1000
пап привет как дела - TUT.BY 909
привет как дела?... Херсон 910
пап привет как дела - Яндекс.Видео 928
привет как дела клип - Яндекс.Видео 929
привет братан как дела - Яндекс.Видео 955
привет как дела клип - TUT.BY 912
привет братан как дела - TUT.BY 944
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
привет 1000
пап привет как дела - TUT.BY 667
привет как дела?... Херсон 672
пап привет как дела - Яндекс.Видео 735
привет как дела клип - Яндекс.Видео 741
привет братан как дела - Яндекс.Видео 753
привет как дела клип - TUT.BY 676
привет братан как дела - TUT.BY 694
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://top.bigmir.net/ 1000
привет 1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://metrica.yandex.com/ 704
http://autometric.ru/ 727
http://metris.ru/ 778
http://metrika.ru/ 789
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 684
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://autometric.ru/ 727
http://metrica.yandex.com/ 778
http://metris.ru/ 778
http://metrika.ru/ 789
1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 769
http://top.bigmir.net/ 727
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000

View File

@ -32,7 +32,7 @@ select round(1000 * ngramDistanceUTF8('абвгдеёжз', 'ёёёёёёёё'))
drop table if exists test_distance;
create table test_distance (Title String) engine = Memory;
insert into test_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
insert into test_distance values ('привет как дела?... Херсон'), ('привет как дела клип - TUT.BY'), ('привет'), ('пап привет как дела - TUT.BY'), ('привет братан как дела - TUT.BY'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://top.bigmir.net/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, Title) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'как дела')) as distance, Title;
@ -44,7 +44,7 @@ SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUT
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'yandex') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'bigmir') as distance, Title;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
@ -91,8 +91,8 @@ SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCa
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'YanDEX') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'BigMIR') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - TuT.by') as distance, Title;
select round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5;
@ -134,7 +134,7 @@ SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(T
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'yandex') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'bigmir') as distance, Title;
select round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
@ -175,6 +175,6 @@ SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCa
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'YanDEX') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'BigMIR') as distance, Title;
drop table if exists test_distance;

View File

@ -600,16 +600,16 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
0
0
0
0
0
0
0
0
0
0
1
1
1

View File

@ -75,18 +75,18 @@ select 1 = multiMatchAny(materialize('abcdef'), ['a......', 'a.....']) from syst
select 0 = multiMatchAny(materialize('aaaa'), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']) from system.numbers limit 10;
select 1 = multiMatchAny(materialize('abc'), ['a\0d']) from system.numbers limit 10;
select 1 = multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['google', 'yandex1']) from system.numbers limit 10;
select 2 = multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['google1', 'yandex']) from system.numbers limit 10;
select 0 != multiMatchAnyIndex(materialize('gogleuedeyandexgoogle'), ['.*goo.*', '.*yan.*']) from system.numbers limit 10;
select 1 = multiMatchAnyIndex(materialize('gogleuedeuniangoogle'), ['google', 'unian1']) from system.numbers limit 10;
select 2 = multiMatchAnyIndex(materialize('gogleuedeuniangoogle'), ['google1', 'unian']) from system.numbers limit 10;
select 0 != multiMatchAnyIndex(materialize('gogleuedeuniangoogle'), ['.*goo.*', '.*yan.*']) from system.numbers limit 10;
select 5 = multiMatchAnyIndex(materialize('vladizlvav dabe don\'t heart me no more'), ['what', 'is', 'love', 'baby', 'no mo??', 'dont', 'h.rt me']) from system.numbers limit 10;
SELECT multiMatchAny(materialize('/odezhda-dlya-bega/'), ['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega']);
SELECT 1 = multiMatchAny('фабрикант', ['f[ae]b[ei]rl', 'ф[иаэе]б[еэи][рпл]', 'афиукд', 'a[ft],th', '^ф[аиеэ]?б?[еэи]?$', 'берлик', 'fab', 'фа[беьв]+е?[рлко]']);
-- All indices tests
SELECT [1, 2] = arraySort(multiMatchAllIndices(materialize('gogleuedeyandexgoogle'), ['.*goo.*', '.*yan.*'])) from system.numbers limit 5;
SELECT [1, 3] = arraySort(multiMatchAllIndices(materialize('gogleuedeyandexgoogle'), ['.*goo.*', 'neverexisted', '.*yan.*'])) from system.numbers limit 5;
SELECT [] = multiMatchAllIndices(materialize('gogleuedeyandexgoogle'), ['neverexisted', 'anotherone', 'andanotherone']) from system.numbers limit 5;
SELECT [1, 2] = arraySort(multiMatchAllIndices(materialize('gogleuedeuniangoogle'), ['.*goo.*', '.*yan.*'])) from system.numbers limit 5;
SELECT [1, 3] = arraySort(multiMatchAllIndices(materialize('gogleuedeuniangoogle'), ['.*goo.*', 'neverexisted', '.*yan.*'])) from system.numbers limit 5;
SELECT [] = multiMatchAllIndices(materialize('gogleuedeuniangoogle'), ['neverexisted', 'anotherone', 'andanotherone']) from system.numbers limit 5;
SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices('фабрикант', ['', 'рикан', 'а', 'f[ae]b[ei]rl', 'ф[иаэе]б[еэи][рпл]', 'афиукд', 'a[ft],th', '^ф[аиеэ]?б?[еэи]?$', 'берлик', 'fab', 'фа[беьв]+е?[рлко]']));
SELECT [1] = multiMatchAllIndices(materialize('/odezhda-dlya-bega/'), ['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega']);
SELECT [] = multiMatchAllIndices(materialize('aaaa'), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']);

View File

@ -113,113 +113,113 @@
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 308
привет братан как дела - Яндекс.Видео 923
пап привет как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 923
пап привет как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 308
привет братан как дела - Яндекс.Видео 769
привет как дела клип - Яндекс.Видео 769
привет братан как дела - TUT.BY 769
привет как дела клип - TUT.BY 769
привет как дела?... Херсон 769
пап привет как дела - Яндекс.Видео 846
пап привет как дела - TUT.BY 846
0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://top.bigmir.net/ 200
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 800
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 800
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
http://metrika.ru/ 250
http://top.bigmir.net/ 1000
1000
1000
1000
@ -335,135 +335,135 @@ http://metrica.yandex.com/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
http://top.bigmir.net/ 1000
пап привет как дела - TUT.BY 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 308
привет братан как дела - Яндекс.Видео 923
пап привет как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 923
пап привет как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 308
привет братан как дела - Яндекс.Видео 769
привет как дела клип - Яндекс.Видео 769
привет братан как дела - TUT.BY 769
привет как дела клип - TUT.BY 769
привет как дела?... Херсон 769
пап привет как дела - Яндекс.Видео 846
пап привет как дела - TUT.BY 846
0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://top.bigmir.net/ 200
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 1000
0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://top.bigmir.net/ 200
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 800
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 800
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
http://metrika.ru/ 250
http://top.bigmir.net/ 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 121
привет как дела?... Херсон 394
привет братан как дела - Яндекс.Видео 788
пап привет как дела - Яндекс.Видео 818
привет как дела клип - Яндекс.Видео 1000
http://metrika.ru/ 32
привет 129
http://top.bigmir.net/ 258
привет как дела?... Херсон 419
привет братан как дела - TUT.BY 452
пап привет как дела - TUT.BY 484
привет как дела клип - TUT.BY 677
1000
1000
1000
@ -579,80 +579,80 @@ http://metris.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 360
привет братан как дела - Яндекс.Видео 960
пап привет как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет братан как дела - TUT.BY 960
пап привет как дела - TUT.BY 1000
привет как дела клип - TUT.BY 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 360
привет братан как дела - Яндекс.Видео 880
привет как дела клип - Яндекс.Видео 880
привет братан как дела - TUT.BY 880
привет как дела клип - TUT.BY 880
привет как дела?... Херсон 880
пап привет как дела - Яндекс.Видео 920
пап привет как дела - TUT.BY 920
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 750
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 750
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
http://top.bigmir.net/ 1000
1000
1000
1000
@ -768,88 +768,88 @@ http://metrica.yandex.com/ 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 0
привет братан как дела - Яндекс.Видео 80
пап привет как дела - Яндекс.Видео 120
привет как дела клип - Яндекс.Видео 120
привет братан как дела - TUT.BY 80
пап привет как дела - TUT.BY 120
привет как дела клип - TUT.BY 120
привет как дела?... Херсон 120
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
http://top.bigmir.net/ 0
привет 0
пап привет как дела - Яндекс.Видео 440
привет братан как дела - Яндекс.Видео 440
привет как дела клип - Яндекс.Видео 440
пап привет как дела - TUT.BY 440
привет братан как дела - TUT.BY 440
привет как дела клип - TUT.BY 440
привет как дела?... Херсон 440
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 1000
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 750
0
пап привет как дела - Яндекс.Видео 0
http://top.bigmir.net/ 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 750
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
пап привет как дела - TUT.BY 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет братан как дела - TUT.BY 0
привет как дела клип - TUT.BY 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
http://top.bigmir.net/ 1000

View File

@ -32,7 +32,7 @@ select round(1000 * ngramSearchUTF8('абвгдеёжз', 'ёёёёёёёё'));
drop table if exists test_entry_distance;
create table test_entry_distance (Title String) engine = Memory;
insert into test_entry_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
insert into test_entry_distance values ('привет как дела?... Херсон'), ('привет как дела клип - TUT.BY'), ('привет'), ('пап привет как дела - TUT.BY'), ('привет братан как дела - TUT.BY'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://top.bigmir.net/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, Title) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, extract(Title, 'как дела')) as distance, Title;
@ -44,7 +44,7 @@ SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSear
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'yandex') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'bigmir') as distance, Title;
select round(1000 * ngramSearchCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
@ -91,8 +91,8 @@ SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSear
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'YanDEX') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'BigMIR') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - bigMir.Net') as distance, Title;
select round(1000 * ngramSearch(materialize(''), '')) from system.numbers limit 5;
@ -134,7 +134,7 @@ SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSear
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'yandex') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'bigmir') as distance, Title;
select round(1000 * ngramSearchCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramSearchCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
@ -175,6 +175,6 @@ SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSear
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'YanDEX') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'BigMIR') as distance, Title;
drop table if exists test_entry_distance;

View File

@ -1 +1 @@
SELECT yandexConsistentHash(-1, 40000); -- { serverError 36 }
SELECT kostikConsistentHash(-1, 40000); -- { serverError 36 }

View File

@ -10,5 +10,5 @@
0
0
3
CREATE TABLE default.constrained\n(\n `URL` String,\n CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = \'yandex.ru\',\n CONSTRAINT is_utf8 CHECK isValidUTF8(URL)\n)\nENGINE = Log
CREATE TABLE default.constrained2\n(\n `URL` String,\n CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = \'yandex.ru\',\n CONSTRAINT is_utf8 CHECK isValidUTF8(URL)\n)\nENGINE = Log
CREATE TABLE default.constrained\n(\n `URL` String,\n CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = \'censor.net\',\n CONSTRAINT is_utf8 CHECK isValidUTF8(URL)\n)\nENGINE = Log
CREATE TABLE default.constrained2\n(\n `URL` String,\n CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = \'censor.net\',\n CONSTRAINT is_utf8 CHECK isValidUTF8(URL)\n)\nENGINE = Log

View File

@ -1,53 +1,53 @@
DROP TABLE IF EXISTS constrained;
CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Null;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst'));
CREATE TABLE constrained (URL String, CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = 'censor.net', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Null;
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), ('https://censor.net/te\xFFst'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), (toValidUTF8('https://censor.net/te\xFFst'));
DROP TABLE constrained;
CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Memory;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
CREATE TABLE constrained (URL String, CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = 'censor.net', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Memory;
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), ('https://censor.net/te\xFFst'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst'));
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), (toValidUTF8('https://censor.net/te\xFFst'));
SELECT count() FROM constrained;
DROP TABLE constrained;
CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = StripeLog;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
CREATE TABLE constrained (URL String, CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = 'censor.net', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = StripeLog;
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), ('https://censor.net/te\xFFst'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst'));
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), (toValidUTF8('https://censor.net/te\xFFst'));
SELECT count() FROM constrained;
DROP TABLE constrained;
CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = TinyLog;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
CREATE TABLE constrained (URL String, CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = 'censor.net', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = TinyLog;
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), ('https://censor.net/te\xFFst'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst'));
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), (toValidUTF8('https://censor.net/te\xFFst'));
SELECT count() FROM constrained;
DROP TABLE constrained;
CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
CREATE TABLE constrained (URL String, CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = 'censor.net', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log;
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), ('https://yandex.ru/te\xFFst'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), ('https://censor.net/te\xFFst'); -- { serverError 469 }
SELECT count() FROM constrained;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('ftp://yandex.ru/Hello'), (toValidUTF8('https://yandex.ru/te\xFFst'));
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('ftp://censor.net/Hello'), (toValidUTF8('https://censor.net/te\xFFst'));
SELECT count() FROM constrained;
DROP TABLE constrained;
DROP TABLE IF EXISTS constrained2;
CREATE TABLE constrained (URL String, CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log;
CREATE TABLE constrained (URL String, CONSTRAINT is_censor CHECK domainWithoutWWW(URL) = 'censor.net', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = Log;
CREATE TABLE constrained2 AS constrained;
SHOW CREATE TABLE constrained;
SHOW CREATE TABLE constrained2;
INSERT INTO constrained VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
INSERT INTO constrained2 VALUES ('https://www.yandex.ru/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
INSERT INTO constrained VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
INSERT INTO constrained2 VALUES ('https://www.censor.net/?q=upyachka'), ('Hello'), ('test'); -- { serverError 469 }
DROP TABLE constrained;
DROP TABLE constrained2;

View File

@ -5,7 +5,7 @@ set allow_experimental_parallel_reading_from_replicas=0;
drop table if exists test_max_parallel_replicas_lr;
-- If you wonder why the table is named with "_lr" suffix in this test.
-- No reason. Actually it is the name of the table in Yandex.Market and they provided this test case for us.
-- No reason. Actually it is the name of the table in our customer and they provided this test case for us.
CREATE TABLE test_max_parallel_replicas_lr (timestamp UInt64) ENGINE = MergeTree ORDER BY (intHash32(timestamp)) SAMPLE BY intHash32(timestamp);
INSERT INTO test_max_parallel_replicas_lr select number as timestamp from system.numbers limit 100;

View File

@ -22,3 +22,4 @@ for m in gz br xz zst lz4 bz2
do
${CLICKHOUSE_CLIENT} --query "SELECT count() < 4000000, max(x) FROM file('${CLICKHOUSE_DATABASE}/${m}.tsv.${m}', RowBinary, 'x UInt8', 'none')"
done

View File

@ -1,5 +1,7 @@
-- Tags: no-parallel, no-fasttest
SET prefer_localhost_replica=1;
DROP TABLE IF EXISTS file;
DROP TABLE IF EXISTS url;
DROP TABLE IF EXISTS view;

View File

@ -3,6 +3,7 @@
-- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971
SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525
SET prefer_localhost_replica = 1;
DROP TABLE IF EXISTS local_01099_a;
DROP TABLE IF EXISTS local_01099_b;

View File

@ -1,4 +1,4 @@
-- Tags: distributed, no-replicated-database, no-parallel, no-fasttest
-- Tags: distributed, no-replicated-database, no-parallel, no-fasttest, no-random-settings
SET allow_experimental_live_view = 1;

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings
# Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently
# Regression for MemoryTracker that had been incorrectly accounted

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, zookeeper, no-parallel
# Tags: long, zookeeper, no-parallel, no-backward-compatibility-check
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -2,13 +2,13 @@ other
other
google
other
yandex
censor.net
other
yahoo
other
other
other
SELECT transform(number, [2, 4, 6], _CAST([\'google\', \'yandex\', \'yahoo\'], \'Array(Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4))\'), _CAST(\'other\', \'Enum8(\\\'google\\\' = 1, \\\'other\\\' = 2, \\\'yahoo\\\' = 3, \\\'yandex\\\' = 4)\'))
SELECT transform(number, [2, 4, 6], _CAST([\'google\', \'censor.net\', \'yahoo\'], \'Array(Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4))\'), _CAST(\'other\', \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\'))
FROM system.numbers
LIMIT 10
google
@ -17,24 +17,24 @@ google
google
google
google
yandex
yandex
yandex
yandex
SELECT if(number > 5, \'yandex\', \'google\')
censor.net
censor.net
censor.net
censor.net
SELECT if(number > 5, \'censor.net\', \'google\')
FROM system.numbers
LIMIT 10
other
other
google
other
yandex
censor.net
other
yahoo
other
other
other
SELECT transform(number, [2, 4, 6], [\'google\', \'yandex\', \'yahoo\'], \'other\')
SELECT transform(number, [2, 4, 6], [\'google\', \'censor.net\', \'yahoo\'], \'other\')
FROM system.numbers
LIMIT 10
google
@ -43,10 +43,10 @@ google
google
google
google
yandex
yandex
yandex
yandex
SELECT if(number > 5, \'yandex\', \'google\')
censor.net
censor.net
censor.net
censor.net
SELECT if(number > 5, \'censor.net\', \'google\')
FROM system.numbers
LIMIT 10

View File

@ -1,13 +1,13 @@
set optimize_if_transform_strings_to_enum = 1;
SELECT transform(number, [2, 4, 6], ['google', 'yandex', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT transform(number, [2, 4, 6], ['google', 'yandex', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
SELECT number > 5 ? 'yandex' : 'google' FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT number > 5 ? 'yandex' : 'google' FROM system.numbers LIMIT 10;
SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
SELECT number > 5 ? 'censor.net' : 'google' FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT number > 5 ? 'censor.net' : 'google' FROM system.numbers LIMIT 10;
set optimize_if_transform_strings_to_enum = 0;
SELECT transform(number, [2, 4, 6], ['google', 'yandex', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT transform(number, [2, 4, 6], ['google', 'yandex', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
SELECT number > 5 ? 'yandex' : 'google' FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT number > 5 ? 'yandex' : 'google' FROM system.numbers LIMIT 10;
SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') FROM system.numbers LIMIT 10;
SELECT number > 5 ? 'censor.net' : 'google' FROM system.numbers LIMIT 10;
EXPLAIN SYNTAX SELECT number > 5 ? 'censor.net' : 'google' FROM system.numbers LIMIT 10;

View File

@ -1,7 +1,7 @@
CREATE TEMPORARY TABLE constrained
(
`URL` String,
CONSTRAINT is_yandex CHECK domainWithoutWWW(URL) = domainWithoutWWW(URL),
CONSTRAINT identity CHECK domainWithoutWWW(URL) = domainWithoutWWW(URL),
CONSTRAINT is_utf8 CHECK isValidUTF8(URL)
);

View File

@ -1,6 +1,5 @@
SELECT arrayFilter((a) -> ((a, arrayJoin([])) IN (Null, [Null])), []);
SELECT arrayFilter((a) -> ((a, arrayJoin([[]])) IN (Null, [Null])), []);
-- simplified from the https://clickhouse-test-reports.s3.yandex.net/10373/6c4748a63e7acde2cc3283d96ffec590aae1e724/fuzzer/fuzzer.log#fail1
SELECT * FROM system.one ARRAY JOIN arrayFilter((a) -> ((a, arrayJoin([])) IN (NULL)), []) AS arr_x; -- { serverError 43; }
SELECT * FROM numbers(1) LEFT ARRAY JOIN arrayFilter((x_0, x_1) -> (arrayJoin([]) IN (NULL)), [], []) AS arr_x;

View File

@ -8,8 +8,8 @@ SET optimize_if_transform_strings_to_enum = 1;
SELECT any(number + 1) FROM numbers(1);
SELECT uniq(bitNot(number)) FROM numbers(1);
SELECT sum(number + 1) FROM numbers(1);
SELECT transform(number, [1, 2], ['google', 'yandex'], 'other') FROM numbers(1);
SELECT number > 0 ? 'yandex' : 'google' FROM numbers(1);
SELECT transform(number, [1, 2], ['google', 'censor.net'], 'other') FROM numbers(1);
SELECT number > 0 ? 'censor.net' : 'google' FROM numbers(1);
DROP TABLE IF EXISTS local_table;
@ -23,8 +23,8 @@ INSERT INTO local_table SELECT number FROM numbers(1);
SELECT any(number + 1) FROM dist;
SELECT uniq(bitNot(number)) FROM dist;
SELECT sum(number + 1) FROM dist;
SELECT transform(number, [1, 2], ['google', 'yandex'], 'other') FROM dist;
SELECT number > 0 ? 'yandex' : 'google' FROM dist;
SELECT transform(number, [1, 2], ['google', 'censor.net'], 'other') FROM dist;
SELECT number > 0 ? 'censor.net' : 'google' FROM dist;
DROP TABLE local_table;
DROP TABLE dist;

View File

@ -1,3 +1,5 @@
-- Tags: no-random-settings
DROP TABLE IF EXISTS buf_dest;
DROP TABLE IF EXISTS buf;

View File

@ -1,6 +1,7 @@
DROP TABLE IF EXISTS select_final;
SET do_not_merge_across_partitions_select_final = 1;
SET max_threads = 0;
CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t);

View File

@ -1,4 +1,4 @@
-- Tags: no-parallel, no-fasttest
-- Tags: no-parallel, no-fasttest, no-random-settings
SET max_memory_usage = 1, max_untracked_memory = 1000000, max_threads=40;
select 'test', count(*) from zeros_mt(1000000) where not ignore(zero); -- { serverError 241 }

View File

@ -8,23 +8,23 @@ SET optimize_move_to_prewhere = 1;
SET optimize_substitute_columns = 1;
SET optimize_append_index = 1;
CREATE TABLE constraint_test_assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog;
CREATE TABLE constraint_test_assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'bigmir.net', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog;
--- Add wrong rows in order to check optimization
INSERT INTO constraint_test_assumption (URL, a) VALUES ('1', 1);
INSERT INTO constraint_test_assumption (URL, a) VALUES ('2', 2);
INSERT INTO constraint_test_assumption (URL, a) VALUES ('yandex.ru', 3);
INSERT INTO constraint_test_assumption (URL, a) VALUES ('bigmir.net', 3);
INSERT INTO constraint_test_assumption (URL, a) VALUES ('3', 4);
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'yandex.ru'; --- assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE NOT (domainWithoutWWW(URL) = 'yandex.ru'); --- assumption -> 0
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- assumption -> 0
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'bigmir.net'; --- assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE NOT (domainWithoutWWW(URL) = 'bigmir.net'); --- assumption -> 0
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) != 'bigmir.net'; --- assumption -> 0
SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz'); ---> assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND NOT URL <= 'zzz'); ---> assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'bigmir.net' AND URL > 'zzz'); ---> assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'bigmir.net' AND NOT URL <= 'zzz'); ---> assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'bigmir.net' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4
SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'bigmir.net' AND URL = '111'); ---> assumption & no assumption -> 0
SELECT count() FROM constraint_test_assumption WHERE (startsWith(URL, 'test') = True); ---> assumption -> 4
DROP TABLE constraint_test_assumption;

View File

@ -1,2 +1,2 @@
1
https://yandex.ru/
https://meta.ua/

View File

@ -4,6 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT 1' --referer 'https://yandex.ru/'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT 1' --referer 'https://meta.ua/'
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} --query "SELECT http_referer FROM system.query_log WHERE current_database = currentDatabase() AND http_referer LIKE '%yandex%' LIMIT 1"
${CLICKHOUSE_CLIENT} --query "SELECT http_referer FROM system.query_log WHERE current_database = currentDatabase() AND http_referer LIKE '%meta%' LIMIT 1"

View File

@ -16,15 +16,3 @@ done
#echo "I = ${I}"
echo 'Ok'
counter=0
I=0
while [[ $counter -lt $retries ]]; do
I=$((I + 1))
TYPE=$(perl -e "print 'Array(' x $I; print 'UInt8'; print ')' x $I")
${CLICKHOUSE_CLIENT} --prefer_localhost_replica=0 --max_parser_depth 1000000 --query "SELECT * FROM remote('127.0.0.{1,2}', generateRandom('x $TYPE', 1, 1, 1)) LIMIT 1 FORMAT Null" 2>&1 | grep -q -F 'Maximum parse depth' && break;
((++counter))
done
#echo "I = ${I}"
echo 'Ok'

View File

@ -6,6 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()";
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
${CLICKHOUSE_CLIENT} -q "drop table insert_big_json"

View File

@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON SETTINGS max_block_size=65505" | brotli -d | tail -n30 | head -n23

View File

@ -1,4 +1,4 @@
-- Tags: long, distributed
-- Tags: long, distributed, no-random-settings
drop table if exists data_01730;

View File

@ -1,4 +1,4 @@
SELECT arrayStringConcat(arrayMap(x -> transform(x, [1025, -9223372036854775808, 65537, 257, 1048576, 10, 7, 1048575, 65536], ['yandex', 'googlegooglegooglegoogle', 'test', '', '', 'hello', 'world', '', 'xyz'], ''), arrayMap(x -> (x % -inf), range(number))), '')
SELECT arrayStringConcat(arrayMap(x -> transform(x, [1025, -9223372036854775808, 65537, 257, 1048576, 10, 7, 1048575, 65536], ['censor.net', 'googlegooglegooglegoogle', 'test', '', '', 'hello', 'world', '', 'xyz'], ''), arrayMap(x -> (x % -inf), range(number))), '')
FROM system.numbers
LIMIT 1025
FORMAT Null;

View File

@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON SETTINGS max_block_size=65505" | gzip -d | tail -n30 | head -n23

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tags: no-fasttest, no-random-settings
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,8 +1,8 @@
['It','is','quite','a','wonderful','day','isn','t','it']
['There','is','so','much','to','learn']
['22','00','email','yandex','ru']
['22','00','email','tut','by']
['Токенизация','каких','либо','других','языков']
['It','is','quite','a','wonderful','day,','isn\'t','it?']
['There','is....','so','much','to','learn!']
['22:00','email@yandex.ru']
['22:00','email@tut.by']
['Токенизация','каких-либо','других','языков?']

View File

@ -2,10 +2,10 @@ SET allow_experimental_nlp_functions = 1;
SELECT splitByNonAlpha('It is quite a wonderful day, isn\'t it?');
SELECT splitByNonAlpha('There is.... so much to learn!');
SELECT splitByNonAlpha('22:00 email@yandex.ru');
SELECT splitByNonAlpha('22:00 email@tut.by');
SELECT splitByNonAlpha('Токенизация каких-либо других языков?');
SELECT splitByWhitespace('It is quite a wonderful day, isn\'t it?');
SELECT splitByWhitespace('There is.... so much to learn!');
SELECT splitByWhitespace('22:00 email@yandex.ru');
SELECT splitByWhitespace('22:00 email@tut.by');
SELECT splitByWhitespace('Токенизация каких-либо других языков?');

View File

@ -12,6 +12,7 @@ mkdir -p ${user_files_path}/
cp $CUR_DIR/data_zstd/test_01946.zstd ${user_files_path}/
${CLICKHOUSE_CLIENT} --multiline --multiquery --query "
set min_chunk_bytes_for_parallel_parsing=10485760;
set max_read_buffer_size = 65536;
set input_format_parallel_parsing = 0;
select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') limit 30 format Null;

View File

@ -66,8 +66,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# frame #11: 0x000000000fffdfc4 clickhouse`main(argc_=<unavailable>, argv_=<unavailable>) + 356 at main.cpp:366
# frame #12: 0x00007ffff7de6d0a libc.so.6`__libc_start_main(main=(clickhouse`main at main.cpp:339), argc=7, argv=0x00007fffffffe1e8, init=<unavailable>, fini=<unavailable>, rtld_fini=<unavailable>, stack_end=0x00007fffffffe1d8) + 234 at libc-start.c:308
# frame #13: 0x000000000ffdc0aa clickhouse`_start + 42
#
# [1]: https://clickhouse-test-reports.s3.yandex.net/26656/f17ca450ac991603e6400c7caef49c493ac69739/functional_stateless_tests_(ubsan).html#fail1
# Limit number of files to 50, and we will get EMFILE for some of socket()
prlimit --nofile=50 $CLICKHOUSE_BENCHMARK --iterations 1 --concurrency 50 --query 'select 1' 2>&1

View File

@ -1,13 +1,14 @@
#!/usr/bin/env bash
# Tags: no-random-settings
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1"
url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1&max_insert_threads=0&group_by_two_level_threshold=100000&group_by_two_level_threshold_bytes=50000000&distributed_aggregation_memory_efficient=1&fsync_metadata=1&priority=1&output_format_parallel_formatting=0&input_format_parallel_parsing=0&min_chunk_bytes_for_parallel_parsing=4031398&max_read_buffer_size=554729&prefer_localhost_replica=0&max_block_size=51672&max_threads=20"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "DROP TABLE IF EXISTS async_inserts"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id"
${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV
1,"a"
@ -22,7 +23,7 @@ ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV
wait
${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts ORDER BY id"
${CLICKHOUSE_CLIENT} -q "SELECT name, rows, level FROM system.parts WHERE table = 'async_inserts' AND database = '$CLICKHOUSE_DATABASE' ORDER BY name"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "SELECT * FROM async_inserts ORDER BY id"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "SELECT name, rows, level FROM system.parts WHERE table = 'async_inserts' AND database = '$CLICKHOUSE_DATABASE' ORDER BY name"
${CLICKHOUSE_CLIENT} -q "DROP TABLE async_inserts"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "DROP TABLE async_inserts"

View File

@ -29,5 +29,5 @@ $CLICKHOUSE_CLIENT -q "create table test_dist engine=Distributed('test_shard_loc
$CLICKHOUSE_CLIENT -q "detach table test_dist"
$CLICKHOUSE_CLIENT -q "drop table test"
$CLICKHOUSE_CLIENT -q "attach table test_dist"
$CLICKHOUSE_CLIENT -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL"
$CLICKHOUSE_CLIENT --prefer_localhost_replica=1 -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL"

View File

@ -1,4 +1,4 @@
-- Tags: replica, distributed
-- Tags: replica, distributed, no-random-settings
SET max_parallel_replicas = 2;
SELECT EventTime::DateTime('Asia/Dubai') FROM remote('127.0.0.{1|2}', test, hits) ORDER BY EventTime DESC LIMIT 10

View File

@ -13,9 +13,9 @@ do
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()"
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
$CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
@ -23,9 +23,9 @@ do
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()"
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
$CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"