mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge branch 'master' into libcxx14
This commit is contained in:
commit
0b95348215
2
contrib/libxml2
vendored
2
contrib/libxml2
vendored
@ -1 +1 @@
|
||||
Subproject commit 18890f471c420411aa3c989e104d090966ec9dbf
|
||||
Subproject commit a075d256fd9ff15590b86d981b75a50ead124fca
|
@ -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 │ │
|
||||
|
@ -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(
|
||||
|
@ -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>;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
49
src/Common/GetPriorityForLoadBalancing.cpp
Normal file
49
src/Common/GetPriorityForLoadBalancing.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
34
src/Common/GetPriorityForLoadBalancing.h
Normal file
34
src/Common/GetPriorityForLoadBalancing.h
Normal 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.
|
||||
};
|
||||
|
||||
}
|
@ -22,7 +22,6 @@ target_link_libraries (clickhouse_common_zookeeper_no_log
|
||||
PRIVATE
|
||||
string_utils
|
||||
)
|
||||
|
||||
if (ENABLE_EXAMPLES)
|
||||
add_subdirectory(examples)
|
||||
endif()
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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])
|
||||
|
@ -149,4 +149,5 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation , ErrorCodes::BAD_ARGUMENTS,
|
||||
{"str", FormatSettings::MsgPackUUIDRepresentation::STR},
|
||||
{"ext", FormatSettings::MsgPackUUIDRepresentation::EXT}})
|
||||
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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>();
|
||||
|
@ -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())
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
|
||||
|
39
src/Interpreters/threadPoolCallbackRunner.cpp
Normal file
39
src/Interpreters/threadPoolCallbackRunner.cpp
Normal 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();
|
||||
});
|
||||
};
|
||||
}
|
||||
|
||||
}
|
15
src/Interpreters/threadPoolCallbackRunner.h
Normal file
15
src/Interpreters/threadPoolCallbackRunner.h
Normal 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);
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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>
|
||||
|
@ -1459,7 +1459,7 @@
|
||||
"xor"
|
||||
"xxHash32"
|
||||
"xxHash64"
|
||||
"yandexConsistentHash"
|
||||
"kostikConsistentHash"
|
||||
"YEAR"
|
||||
"yearweek"
|
||||
"yesterday"
|
||||
|
@ -26,7 +26,7 @@
|
||||
"toUnixTimestamp64Nano"
|
||||
"toUnixTimestamp64Micro"
|
||||
"jumpConsistentHash"
|
||||
"yandexConsistentHash"
|
||||
"kostikConsistentHash"
|
||||
"addressToSymbol"
|
||||
"toJSONString"
|
||||
"JSON_VALUE"
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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>
|
427
tests/integration/test_zookeeper_config_load_balancing/test.py
Normal file
427
tests/integration/test_zookeeper_config_load_balancing/test.py
Normal 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)
|
@ -3,7 +3,7 @@
|
||||
<substitution>
|
||||
<name>hash_func</name>
|
||||
<values>
|
||||
<value>yandexConsistentHash</value>
|
||||
<value>kostikConsistentHash</value>
|
||||
<value>jumpConsistentHash</value>
|
||||
</values>
|
||||
</substitution>
|
||||
|
@ -99,6 +99,6 @@ abc
|
||||
1
|
||||
1
|
||||
Остальные
|
||||
Яндекс
|
||||
Bigmir)net
|
||||
Google
|
||||
Остальные
|
||||
|
@ -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 }
|
||||
|
@ -79,6 +79,6 @@ abc
|
||||
1
|
||||
1
|
||||
Остальные
|
||||
Яндекс
|
||||
Meta.ua
|
||||
Google
|
||||
Остальные
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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)], ';');
|
||||
|
@ -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
|
||||
|
@ -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');
|
||||
|
@ -1,3 +1,3 @@
|
||||
canada congo net-domena
|
||||
yandex yandex yandex яндекс yandex
|
||||
meta bigmir yahoo гугл meta
|
||||
canada hello hello canada
|
||||
|
@ -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'),
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tags: no-fasttest, no-random-settings
|
||||
|
||||
# set -x
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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')
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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}']);
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -1 +1 @@
|
||||
SELECT yandexConsistentHash(-1, 40000); -- { serverError 36 }
|
||||
SELECT kostikConsistentHash(-1, 40000); -- { serverError 36 }
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
);
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-random-settings
|
||||
|
||||
DROP TABLE IF EXISTS buf_dest;
|
||||
DROP TABLE IF EXISTS buf;
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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 }
|
||||
|
@ -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;
|
||||
|
@ -1,2 +1,2 @@
|
||||
1
|
||||
https://yandex.ru/
|
||||
https://meta.ua/
|
||||
|
@ -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"
|
||||
|
@ -1,2 +1 @@
|
||||
Ok
|
||||
Ok
|
||||
|
@ -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'
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: long, distributed
|
||||
-- Tags: long, distributed, no-random-settings
|
||||
|
||||
drop table if exists data_01730;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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']
|
||||
['Токенизация','каких-либо','других','языков?']
|
||||
|
@ -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('Токенизация каких-либо других языков?');
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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"
|
||||
|
@ -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"
|
||||
|
||||
|
@ -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
|
||||
|
@ -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"
|
||||
|
Loading…
Reference in New Issue
Block a user