Merge master

This commit is contained in:
kssenii 2022-03-24 20:47:43 +01:00
commit 6769b84638
134 changed files with 2975 additions and 1336 deletions

2
contrib/libxml2 vendored

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

View File

@ -131,9 +131,6 @@ function start()
# use root to match with current uid
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>>/var/log/clickhouse-server/stderr.log
sleep 0.5
cat /var/log/clickhouse-server/stdout.log
tail -n200 /var/log/clickhouse-server/stderr.log
tail -n200 /var/log/clickhouse-server/clickhouse-server.log
counter=$((counter + 1))
done
@ -211,14 +208,12 @@ stop
start
clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_output/test_results.tsv \
|| echo -e 'Server failed to start\tFAIL' >> /test_output/test_results.tsv
|| (echo -e 'Server failed to start (see application_errors.txt)\tFAIL' >> /test_output/test_results.tsv \
&& grep -Fa "<Error>.*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt)
[ -f /var/log/clickhouse-server/clickhouse-server.log ] || echo -e "Server log does not exist\tFAIL"
[ -f /var/log/clickhouse-server/stderr.log ] || echo -e "Stderr log does not exist\tFAIL"
# Print Fatal log messages to stdout
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log*
# Grep logs for sanitizer asserts, crashes and other critical errors
# Sanitizer asserts
@ -235,20 +230,26 @@ zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/
|| echo -e 'No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Logical errors
zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \
&& echo -e 'Logical error thrown (see clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log* > /test_output/logical_errors.txt \
&& echo -e 'Logical error thrown (see clickhouse-server.log or logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'No logical errors\tOK' >> /test_output/test_results.tsv
# Remove file logical_errors.txt if it's empty
[ -s /test_output/logical_errors.txt ] || rm /test_output/logical_errors.txt
# Crash
zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \
&& echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Not crashed\tOK' >> /test_output/test_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \
&& echo -e 'Fatal message in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log* > /test_output/fatal_messages.txt \
&& echo -e 'Fatal message in clickhouse-server.log (see fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Remove file fatal_messages.txt if it's empty
[ -s /test_output/fatal_messages.txt ] || rm /test_output/fatal_messages.txt
zgrep -Fa "########################################" /test_output/* > /dev/null \
&& echo -e 'Killed by signal (output files)\tFAIL' >> /test_output/test_results.tsv
@ -259,12 +260,12 @@ echo -e "Backward compatibility check\n"
echo "Download previous release server"
mkdir previous_release_package_folder
clickhouse-client --query="SELECT version()" | ./download_previous_release && echo -e 'Download script exit code\tOK' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Download script failed\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
clickhouse-client --query="SELECT version()" | ./download_previous_release && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \
|| echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv
if [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
then
echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/backward_compatibility_check_results.tsv
echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/test_results.tsv
stop
# Uninstall current packages
@ -290,8 +291,8 @@ then
mkdir tmp_stress_output
./stress --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \
&& echo -e 'Test script exit code\tOK' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Test script failed\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
&& echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv
rm -rf tmp_stress_output
clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables"
@ -301,8 +302,9 @@ then
# Start new server
configure
start 500
clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Server failed to start\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \
|| (echo -e 'Backward compatibility check: Server failed to start\tFAIL' >> /test_output/test_results.tsv \
&& grep -Fa "<Error>.*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt)
clickhouse-client --query="SELECT 'Server version: ', version()"
@ -312,10 +314,12 @@ then
stop
# Error messages (we should ignore some errors)
echo "Check for Error messages in server log:"
zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "Code: 236. DB::Exception: Cancelled mutating parts" \
-e "REPLICA_IS_ALREADY_ACTIVE" \
-e "REPLICA_IS_ALREADY_EXIST" \
-e "ALL_REPLICAS_LOST" \
-e "DDLWorker: Cannot parse DDL task query" \
-e "RaftInstance: failed to accept a rpc connection due to error 125" \
-e "UNKNOWN_DATABASE" \
@ -328,47 +332,53 @@ then
-e "Code: 1000, e.code() = 111, Connection refused" \
-e "UNFINISHED" \
-e "Renaming unexpected part" \
/var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "<Error>" > /dev/null \
&& echo -e 'Error message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No Error messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
/var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "<Error>" > /test_output/bc_check_error_messages.txt \
&& echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Remove file bc_check_error_messages.txt if it's empty
[ -s /test_output/bc_check_error_messages.txt ] || rm /test_output/bc_check_error_messages.txt
# Sanitizer asserts
zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" /test_output/tmp > /dev/null \
&& echo -e 'Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No sanitizer asserts\tOK' >> /test_output/backward_compatibility_check_results.tsv
&& echo -e 'Backward compatibility check: Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No sanitizer asserts\tOK' >> /test_output/test_results.tsv
rm -f /test_output/tmp
# OOM
zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No OOM messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
&& echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Logical errors
zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Logical error thrown (see clickhouse-server.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No logical errors\tOK' >> /test_output/backward_compatibility_check_results.tsv
echo "Check for Logical errors in server log:"
zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /test_output/bc_check_logical_errors.txt \
&& echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv
# Remove file bc_check_logical_errors.txt if it's empty
[ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt
# Crash
zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Not crashed\tOK' >> /test_output/backward_compatibility_check_results.tsv
&& echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Fatal message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No fatal messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
echo "Check for Fatal message in server log:"
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log > /test_output/bc_check_fatal_messages.txt \
&& echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Remove file bc_check_fatal_messages.txt if it's empty
[ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt
else
echo -e "Failed to download previous release packets\tFAIL" >> /test_output/backward_compatibility_check_results.tsv
echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv
fi
zgrep -Fa "FAIL" /test_output/backward_compatibility_check_results.tsv > /dev/null \
&& echo -e 'Backward compatibility check\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check\tOK' >> /test_output/test_results.tsv
# Put logs into /test_output/
for log_file in /var/log/clickhouse-server/clickhouse-server.log*
do

View File

@ -195,5 +195,6 @@ toc_title: Adopters
| <a href="https://shop.okraina.ru/" class="favicon">ООО «МПЗ Богородский»</a> | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) |
| <a href="https://domclick.ru/" class="favicon">ДомКлик</a> | Real Estate | — | — | — | [Article in Russian, October 2021](https://habr.com/ru/company/domclick/blog/585936/) |
| <a href="https://magenta-technology.ru/sistema-upravleniya-marshrutami-inkassacii-as-strela/" class="favicon">АС "Стрела"</a> | Transportation | — | — | — | [Job posting, Jan 2022](https://vk.com/topic-111905078_35689124?post=3553) |
| <a href="https://piwik.pro/" class="favicon">Piwik PRO</a> | Web Analytics | — | — | — | [Official website, Dec 2018](https://piwik.pro/blog/piwik-pro-clickhouse-faster-efficient-reports/) |
[Original article](https://clickhouse.com/docs/en/introduction/adopters/) <!--hide-->

View File

@ -5,7 +5,7 @@ toc_title: Caches
# Cache Types {#cache-types}
When performing queries, ClichHouse uses different caches.
When performing queries, ClickHouse uses different caches.
Main cache types:

View File

@ -10,7 +10,7 @@ cssmin==0.2.0
future==0.18.2
htmlmin==0.1.12
idna==2.10
Jinja2>=3.0.3
Jinja2==3.0.3
jinja2-highlight==0.6.1
jsmin==3.0.0
livereload==2.6.3

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -13,6 +13,9 @@ Int32 IntervalKind::toAvgSeconds() const
{
switch (kind)
{
case IntervalKind::Nanosecond: return 0; /// fractional parts of seconds have 0 seconds
case IntervalKind::Microsecond: return 0;
case IntervalKind::Millisecond: return 0;
case IntervalKind::Second: return 1;
case IntervalKind::Minute: return 60;
case IntervalKind::Hour: return 3600;
@ -52,6 +55,9 @@ const char * IntervalKind::toKeyword() const
{
switch (kind)
{
case IntervalKind::Nanosecond: return "NANOSECOND";
case IntervalKind::Microsecond: return "MICROSECOND";
case IntervalKind::Millisecond: return "MILLISECOND";
case IntervalKind::Second: return "SECOND";
case IntervalKind::Minute: return "MINUTE";
case IntervalKind::Hour: return "HOUR";
@ -69,6 +75,9 @@ const char * IntervalKind::toLowercasedKeyword() const
{
switch (kind)
{
case IntervalKind::Nanosecond: return "nanosecond";
case IntervalKind::Microsecond: return "microsecond";
case IntervalKind::Millisecond: return "millisecond";
case IntervalKind::Second: return "second";
case IntervalKind::Minute: return "minute";
case IntervalKind::Hour: return "hour";
@ -86,6 +95,12 @@ const char * IntervalKind::toDateDiffUnit() const
{
switch (kind)
{
case IntervalKind::Nanosecond:
return "nanosecond";
case IntervalKind::Microsecond:
return "microsecond";
case IntervalKind::Millisecond:
return "millisecond";
case IntervalKind::Second:
return "second";
case IntervalKind::Minute:
@ -111,6 +126,12 @@ const char * IntervalKind::toNameOfFunctionToIntervalDataType() const
{
switch (kind)
{
case IntervalKind::Nanosecond:
return "toIntervalNanosecond";
case IntervalKind::Microsecond:
return "toIntervalMicrosecond";
case IntervalKind::Millisecond:
return "toIntervalMillisecond";
case IntervalKind::Second:
return "toIntervalSecond";
case IntervalKind::Minute:
@ -136,6 +157,12 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
{
switch (kind)
{
case IntervalKind::Nanosecond:
return "toNanosecond";
case IntervalKind::Microsecond:
return "toMicrosecond";
case IntervalKind::Millisecond:
return "toMillisecond";
case IntervalKind::Second:
return "toSecond";
case IntervalKind::Minute:
@ -162,6 +189,21 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
bool IntervalKind::tryParseString(const std::string & kind, IntervalKind::Kind & result)
{
if ("nanosecond" == kind)
{
result = IntervalKind::Nanosecond;
return true;
}
if ("microsecond" == kind)
{
result = IntervalKind::Microsecond;
return true;
}
if ("millisecond" == kind)
{
result = IntervalKind::Millisecond;
return true;
}
if ("second" == kind)
{
result = IntervalKind::Second;

View File

@ -10,6 +10,9 @@ struct IntervalKind
{
enum Kind
{
Nanosecond,
Microsecond,
Millisecond,
Second,
Minute,
Hour,
@ -61,6 +64,9 @@ struct IntervalKind
/// NOLINTNEXTLINE
#define FOR_EACH_INTERVAL_KIND(M) \
M(Nanosecond) \
M(Microsecond) \
M(Millisecond) \
M(Second) \
M(Minute) \
M(Hour) \

View File

@ -515,6 +515,11 @@ public:
radixSortLSDInternal<false>(arr, size, false, nullptr);
}
static void executeLSD(Element * arr, size_t size, bool reverse)
{
radixSortLSDInternal<false>(arr, size, reverse, nullptr);
}
/** This function will start to sort inplace (modify 'arr')
* but on the last step it will write result directly to the destination
* instead of finishing sorting 'arr'.

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -13,6 +13,9 @@ bool DataTypeInterval::equals(const IDataType & rhs) const
void registerDataTypeInterval(DataTypeFactory & factory)
{
factory.registerSimpleDataType("IntervalNanosecond", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Nanosecond)); });
factory.registerSimpleDataType("IntervalMicrosecond", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Microsecond)); });
factory.registerSimpleDataType("IntervalMillisecond", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Millisecond)); });
factory.registerSimpleDataType("IntervalSecond", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Second)); });
factory.registerSimpleDataType("IntervalMinute", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Minute)); });
factory.registerSimpleDataType("IntervalHour", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Hour)); });

View File

@ -21,6 +21,7 @@
#include <Common/FileCacheFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <IO/ReadBufferFromS3.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
@ -265,31 +266,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();
});
};
bool cache_on_insert = fs::path(path).extension() != ".tmp"
&& write_settings.remote_fs_cache_on_write_operations
@ -304,7 +280,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), blob_name, cache_on_insert ? cache : nullptr);
buf_size, threadPoolCallbackRunner(getThreadPoolWriter()), blob_name, cache_on_insert ? cache : nullptr);
auto create_metadata_callback = [this, path, blob_name, mode] (size_t count)
{

View File

@ -41,6 +41,11 @@ namespace ErrorCodes
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline UInt32 dateTimeIsNotSupported(const char * name)
{
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
{
@ -311,6 +316,133 @@ struct ToStartOfSecondImpl
using FactorTransform = ZeroTransform;
};
struct ToStartOfMillisecondImpl
{
static constexpr auto name = "toStartOfMillisecond";
static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &)
{
// given that scale is 6, scale_multiplier is 1000000
// for DateTime64 value of 123.456789:
// 123456789 - 789 = 123456000
// for DateTime64 value of -123.456789:
// -123456789 - (1000 + (-789)) = -123457000
if (scale_multiplier == 1000)
{
return datetime64;
}
else if (scale_multiplier <= 1000)
{
return datetime64 * (1000 / scale_multiplier);
}
else
{
auto droppable_part_with_sign = DecimalUtils::getFractionalPartWithScaleMultiplier<DateTime64, true>(datetime64, scale_multiplier / 1000);
if (droppable_part_with_sign < 0)
droppable_part_with_sign += scale_multiplier;
return datetime64 - droppable_part_with_sign;
}
}
static inline UInt32 execute(UInt32, const DateLUTImpl &)
{
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfMicrosecondImpl
{
static constexpr auto name = "toStartOfMicrosecond";
static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &)
{
// @see ToStartOfMillisecondImpl
if (scale_multiplier == 1000000)
{
return datetime64;
}
else if (scale_multiplier <= 1000000)
{
return datetime64 * (1000000 / scale_multiplier);
}
else
{
auto droppable_part_with_sign = DecimalUtils::getFractionalPartWithScaleMultiplier<DateTime64, true>(datetime64, scale_multiplier / 1000000);
if (droppable_part_with_sign < 0)
droppable_part_with_sign += scale_multiplier;
return datetime64 - droppable_part_with_sign;
}
}
static inline UInt32 execute(UInt32, const DateLUTImpl &)
{
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfNanosecondImpl
{
static constexpr auto name = "toStartOfNanosecond";
static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &)
{
// @see ToStartOfMillisecondImpl
if (scale_multiplier == 1000000000)
{
return datetime64;
}
else if (scale_multiplier <= 1000000000)
{
return datetime64 * (1000000000 / scale_multiplier);
}
else
{
throw Exception("Illegal type of argument for function " + std::string(name) + ", DateTime64 expected", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
static inline UInt32 execute(UInt32, const DateLUTImpl &)
{
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfFiveMinuteImpl
{
static constexpr auto name = "toStartOfFiveMinute";

View File

@ -40,26 +40,158 @@ namespace ErrorCodes
/// - 'AddSecondsImpl::execute(UInt32, ...) -> UInt32' is available to the ClickHouse users as 'addSeconds(DateTime, ...) -> DateTime'
/// - 'AddSecondsImpl::execute(UInt16, ...) -> UInt32' is available to the ClickHouse users as 'addSeconds(Date, ...) -> DateTime'
struct AddNanosecondsImpl
{
static constexpr auto name = "addNanoseconds";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &, UInt16 scale = DataTypeDateTime64::default_scale)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(9 - scale);
auto division = std::div(t.fractional * multiplier + delta, static_cast<Int64>(1000000000));
return {t.whole * multiplier + division.quot, t.fractional * multiplier + delta};
}
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(9 - scale);
return t * multiplier + delta;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(9);
return t * multiplier + delta;
}
static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0)
{
throw Exception("addNanoSeconds() cannot be used with Date", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline NO_SANITIZE_UNDEFINED DateTime64 execute(Int32, Int64, const DateLUTImpl &, UInt16 = 0)
{
throw Exception("addNanoSeconds() cannot be used with Date32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct AddMicrosecondsImpl
{
static constexpr auto name = "addMicroseconds";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(std::abs(6 - scale));
if (scale <= 6)
{
auto division = std::div((t.fractional + delta), static_cast<Int64>(10e6));
return {t.whole * multiplier + division.quot, division.rem};
}
else
{
auto division = std::div((t.fractional + delta * multiplier), static_cast<Int64>(10e6 * multiplier));
return {t.whole + division.quot, division.rem};
}
}
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(std::abs(6 - scale));
return scale <= 6 ? t * multiplier + delta : t + delta * multiplier;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(6);
return t * multiplier + delta;
}
static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0)
{
throw Exception("addMicroSeconds() cannot be used with Date", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline NO_SANITIZE_UNDEFINED DateTime64 execute(Int32, Int64, const DateLUTImpl &, UInt16 = 0)
{
throw Exception("addMicroSeconds() cannot be used with Date32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct AddMillisecondsImpl
{
static constexpr auto name = "addMilliseconds";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &, UInt16 scale = DataTypeDateTime64::default_scale)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(std::abs(3 - scale));
if (scale <= 3)
{
auto division = std::div((t.fractional + delta), static_cast<Int64>(1000));
return {t.whole * multiplier + division.quot, division.rem};
}
else
{
auto division = std::div((t.fractional + delta * multiplier), static_cast<Int64>(1000 * multiplier));
return {t.whole + division.quot,division.rem};
}
}
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(std::abs(3 - scale));
return scale <= 3 ? t * multiplier + delta : t + delta * multiplier;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(3);
return t * multiplier + delta;
}
static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0)
{
throw Exception("addMilliSeconds() cannot be used with Date", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline NO_SANITIZE_UNDEFINED DateTime64 execute(Int32, Int64, const DateLUTImpl &, UInt16 = 0)
{
throw Exception("addMilliSeconds() cannot be used with Date32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct AddSecondsImpl
{
static constexpr auto name = "addSeconds";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &)
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return {t.whole + delta, t.fractional};
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>(scale);
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return t + delta;
}
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
// use default datetime64 scale
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta) * 1000;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.fromDayNum(DayNum(d)) + delta;
}
@ -70,21 +202,29 @@ struct AddMinutesImpl
static constexpr auto name = "addMinutes";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &)
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return {t.whole + delta * 60, t.fractional};
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
return t + 60 * delta * DecimalUtils::scaleMultiplier<DateTime64>(scale);
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return t + delta * 60;
}
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
// use default datetime64 scale
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60) * 1000;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.fromDayNum(DayNum(d)) + delta * 60;
}
@ -95,20 +235,29 @@ struct AddHoursImpl
static constexpr auto name = "addHours";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &)
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return {t.whole + delta * 3600, t.fractional};
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0)
{
return t + 3600 * delta * DecimalUtils::scaleMultiplier<DateTime64>(scale);
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return t + delta * 3600;
}
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
// use default datetime64 scale
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600) * 1000;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.fromDayNum(DayNum(d)) + delta * 3600;
}
@ -119,22 +268,30 @@ struct AddDaysImpl
static constexpr auto name = "addDays";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return {time_zone.addDays(t.whole, delta), t.fractional};
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addDays(d.quot, delta) * multiplier + d.rem;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addDays(t, delta);
}
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return d + delta;
}
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16 = 0)
{
return d + delta;
}
@ -145,22 +302,30 @@ struct AddWeeksImpl
static constexpr auto name = "addWeeks";
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone)
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return {time_zone.addWeeks(t.whole, delta), t.fractional};
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 scale = 0)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addDays(d.quot, delta * 7) * multiplier + d.rem;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addWeeks(t, delta);
}
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl &, UInt16 = 0)
{
return d + delta * 7;
}
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int32 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int32 delta, const DateLUTImpl &, UInt16 = 0)
{
return d + delta * 7;
}
@ -170,23 +335,31 @@ struct AddMonthsImpl
{
static constexpr auto name = "addMonths";
static inline DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return {time_zone.addMonths(t.whole, delta), t.fractional};
}
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addMonths(d.quot, delta) * multiplier + d.rem;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addMonths(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addMonths(DayNum(d), delta);
}
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addMonths(ExtendedDayNum(d), delta);
}
@ -197,22 +370,30 @@ struct AddQuartersImpl
static constexpr auto name = "addQuarters";
static inline DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone)
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return {time_zone.addQuarters(t.whole, delta), t.fractional};
}
static inline UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 scale = 0)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addQuarters(d.quot, delta) * multiplier + d.rem;
}
static inline UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addQuarters(t, delta);
}
static inline UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl & time_zone)
static inline UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addQuarters(DayNum(d), delta);
}
static inline Int32 execute(Int32 d, Int32 delta, const DateLUTImpl & time_zone)
static inline Int32 execute(Int32 d, Int32 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addQuarters(ExtendedDayNum(d), delta);
}
@ -222,23 +403,31 @@ struct AddYearsImpl
{
static constexpr auto name = "addYears";
static inline DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return {time_zone.addYears(t.whole, delta), t.fractional};
}
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED DateTime64
execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addYears(d.quot, delta) * multiplier + d.rem;
}
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addYears(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addYears(DayNum(d), delta);
}
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0)
{
return time_zone.addYears(ExtendedDayNum(d), delta);
}
@ -250,13 +439,16 @@ struct SubtractIntervalImpl : public Transform
using Transform::Transform;
template <typename T>
inline NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone) const
inline NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
/// Signed integer overflow is Ok.
return Transform::execute(t, -delta, time_zone);
return Transform::execute(t, -delta, time_zone, scale);
}
};
struct SubtractNanosecondsImpl : SubtractIntervalImpl<AddNanosecondsImpl> { static constexpr auto name = "subtractNanoseconds"; };
struct SubtractMicrosecondsImpl : SubtractIntervalImpl<AddMicrosecondsImpl> { static constexpr auto name = "subtractMicroseconds"; };
struct SubtractMillisecondsImpl : SubtractIntervalImpl<AddMillisecondsImpl> { static constexpr auto name = "subtractMilliseconds"; };
struct SubtractSecondsImpl : SubtractIntervalImpl<AddSecondsImpl> { static constexpr auto name = "subtractSeconds"; };
struct SubtractMinutesImpl : SubtractIntervalImpl<AddMinutesImpl> { static constexpr auto name = "subtractMinutes"; };
struct SubtractHoursImpl : SubtractIntervalImpl<AddHoursImpl> { static constexpr auto name = "subtractHours"; };
@ -277,17 +469,17 @@ struct Adder
{}
template <typename FromVectorType, typename ToVectorType>
void NO_INLINE vectorConstant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone) const
void NO_INLINE vectorConstant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta), time_zone);
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta), time_zone, scale);
}
template <typename FromVectorType, typename ToVectorType>
void vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
void vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
size_t size = vec_from.size();
vec_to.resize(size);
@ -296,11 +488,11 @@ struct Adder
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>(
&delta, [&](const auto & column){ vectorVector(vec_from, vec_to, column, time_zone, size); return true; });
&delta, [&](const auto & column){ vectorVector(vec_from, vec_to, column, time_zone, scale, size); return true; });
}
template <typename FromType, typename ToVectorType>
void constantVector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
void constantVector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
size_t size = delta.size();
vec_to.resize(size);
@ -309,7 +501,7 @@ struct Adder
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>(
&delta, [&](const auto & column){ constantVector(from, vec_to, column, time_zone, size); return true; });
&delta, [&](const auto & column){ constantVector(from, vec_to, column, time_zone, scale, size); return true; });
}
private:
@ -325,18 +517,18 @@ private:
template <typename FromVectorType, typename ToVectorType, typename DeltaColumnType>
NO_INLINE NO_SANITIZE_UNDEFINED void vectorVector(
const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const
const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale, size_t size) const
{
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta.getData()[i]), time_zone);
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta.getData()[i]), time_zone, scale);
}
template <typename FromType, typename ToVectorType, typename DeltaColumnType>
NO_INLINE NO_SANITIZE_UNDEFINED void constantVector(
const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const
const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale, size_t size) const
{
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone);
vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, scale);
}
};
@ -344,7 +536,7 @@ private:
template <typename FromDataType, typename ToDataType, typename Transform>
struct DateTimeAddIntervalImpl
{
static ColumnPtr execute(Transform transform, const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type)
static ColumnPtr execute(Transform transform, const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, UInt16 scale = 0)
{
using FromValueType = typename FromDataType::FieldType;
using FromColumnType = typename FromDataType::ColumnType;
@ -363,16 +555,15 @@ struct DateTimeAddIntervalImpl
if (const auto * sources = checkAndGetColumn<FromColumnType>(source_col.get()))
{
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
op.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone);
op.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone, scale);
else
op.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone);
op.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone, scale);
}
else if (const auto * sources_const = checkAndGetColumnConst<FromColumnType>(source_col.get()))
{
op.constantVector(
sources_const->template getValue<FromValueType>(),
col_to->getData(),
delta_column, time_zone);
col_to->getData(), delta_column, time_zone, scale);
}
else
{
@ -463,18 +654,10 @@ public:
}
}
// TransformDateTime64 helps choosing correct overload of exec and does some transformations
// on input and output parameters to simplify support of DateTime64 in concrete Transform.
template <typename FieldType>
using TransformType = std::conditional_t<
std::is_same_v<FieldType, DateTime64>,
TransformDateTime64<Transform>,
Transform>;
/// Helper templates to deduce return type based on argument type, since some overloads may promote or denote types,
/// e.g. addSeconds(Date, 1) => DateTime
template <typename FieldType>
using TransformExecuteReturnType = decltype(std::declval<TransformType<FieldType>>().execute(FieldType(), 0, std::declval<DateLUTImpl>()));
using TransformExecuteReturnType = decltype(std::declval<Transform>().execute(FieldType(), 0, std::declval<DateLUTImpl>(), 0));
// Deduces RETURN DataType from INPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl).
// e.g. for Transform-type that has execute()-overload with 'UInt16' input and 'UInt32' return,
@ -500,11 +683,33 @@ public:
if (typeid_cast<const DataTypeDateTime64 *>(arguments[0].type.get()))
{
const auto & datetime64_type = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type);
return std::make_shared<DataTypeDateTime64>(datetime64_type.getScale(), extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
auto from_scale = datetime64_type.getScale();
auto scale = from_scale;
if (std::is_same_v<Transform, AddNanosecondsImpl>)
scale = 9;
else if (std::is_same_v<Transform, AddMicrosecondsImpl>)
scale = 6;
else if (std::is_same_v<Transform, AddMillisecondsImpl>)
scale = 3;
scale = std::max(scale, from_scale);
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
else
{
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
auto scale = DataTypeDateTime64::default_scale;
if (std::is_same_v<Transform, AddNanosecondsImpl>)
scale = 9;
else if (std::is_same_v<Transform, AddMicrosecondsImpl>)
scale = 6;
else if (std::is_same_v<Transform, AddMillisecondsImpl>)
scale = 3;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
}
else
@ -541,9 +746,9 @@ public:
}
else if (const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(from_type))
{
using WrappedTransformType = TransformType<typename DataTypeDateTime64::FieldType>;
return DateTimeAddIntervalImpl<DataTypeDateTime64, TransformResultDataType<DataTypeDateTime64>, WrappedTransformType>::execute(
WrappedTransformType{datetime64_type->getScale()}, arguments, result_type);
auto from_scale = datetime64_type->getScale();
return DateTimeAddIntervalImpl<DataTypeDateTime64, TransformResultDataType<DataTypeDateTime64>, Transform>::execute(
Transform{}, arguments, result_type, from_scale);
}
else
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),

View File

@ -88,6 +88,20 @@ public:
Int64 scale = DataTypeDateTime64::default_scale;
if (const auto * dt64 = checkAndGetDataType<DataTypeDateTime64>(arguments[0].type.get()))
scale = dt64->getScale();
auto source_scale = scale;
if constexpr (std::is_same_v<ToStartOfMillisecondImpl, Transform>)
{
scale = std::max(source_scale, static_cast<Int64>(3));
}
else if constexpr (std::is_same_v<ToStartOfMicrosecondImpl, Transform>)
{
scale = std::max(source_scale, static_cast<Int64>(6));
}
else if constexpr (std::is_same_v<ToStartOfNanosecondImpl, Transform>)
{
scale = std::max(source_scale, static_cast<Int64>(9));
}
return std::make_shared<ToDataType>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
}

View File

@ -112,6 +112,9 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionParseDateTime64BestEffortOrZero>();
factory.registerFunction<FunctionParseDateTime64BestEffortOrNull>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalNanosecond, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMicrosecond, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMillisecond, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalSecond, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMinute, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalHour, PositiveMonotonicity>>();

View File

@ -1487,6 +1487,9 @@ struct NameToDecimal256 { static constexpr auto name = "toDecimal256"; };
static constexpr auto kind = IntervalKind::INTERVAL_KIND; \
};
DEFINE_NAME_TO_INTERVAL(Nanosecond)
DEFINE_NAME_TO_INTERVAL(Microsecond)
DEFINE_NAME_TO_INTERVAL(Millisecond)
DEFINE_NAME_TO_INTERVAL(Second)
DEFINE_NAME_TO_INTERVAL(Minute)
DEFINE_NAME_TO_INTERVAL(Hour)
@ -2703,13 +2706,10 @@ private:
return createWrapper<ToDataType>(from_type, to_type, requested_result_is_nullable);
}
WrapperType createUInt8ToUInt8Wrapper(const DataTypePtr from_type, const DataTypePtr to_type) const
WrapperType createUInt8ToBoolWrapper(const DataTypePtr from_type, const DataTypePtr to_type) const
{
return [from_type, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) -> ColumnPtr
{
if (isBool(from_type) || !isBool(to_type))
return arguments.front().column;
/// Special case when we convert UInt8 column to Bool column.
/// both columns have type UInt8, but we shouldn't use identity wrapper,
/// because Bool column can contain only 0 and 1.
@ -3506,15 +3506,19 @@ private:
/// 'requested_result_is_nullable' is true if CAST to Nullable type is requested.
WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const
{
bool convert_to_ipv6 = to_type->getCustomName() && to_type->getCustomName()->getName() == "IPv6";
if (isUInt8(from_type) && isBool(to_type))
return createUInt8ToBoolWrapper(from_type, to_type);
if (from_type->equals(*to_type) && !convert_to_ipv6)
{
if (isUInt8(from_type))
return createUInt8ToUInt8Wrapper(from_type, to_type);
/// We can cast IPv6 into IPv6, IPv4 into IPv4, but we should not allow to cast FixedString(16) into IPv6 as part of identity cast
bool safe_convert_custom_types = true;
if (const auto * to_type_custom_name = to_type->getCustomName())
safe_convert_custom_types = from_type->getCustomName() && from_type->getCustomName()->getName() == to_type_custom_name->getName();
else if (const auto * from_type_custom_name = from_type->getCustomName())
safe_convert_custom_types = to_type->getCustomName() && from_type_custom_name->getName() == to_type->getCustomName()->getName();
if (from_type->equals(*to_type) && safe_convert_custom_types)
return createIdentityWrapper(from_type);
}
else if (WhichDataType(from_type).isNothing())
return createNothingWrapper(to_type.get());

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int SYNTAX_ERROR;
}
namespace
@ -167,6 +168,13 @@ struct TimeWindowImpl<TUMBLE>
switch (std::get<0>(interval))
{
//TODO: add proper support for fractional seconds
// case IntervalKind::Nanosecond:
// return executeTumble<UInt32, IntervalKind::Nanosecond>(*time_column_vec, std::get<1>(interval), time_zone);
// case IntervalKind::Microsecond:
// return executeTumble<UInt32, IntervalKind::Microsecond>(*time_column_vec, std::get<1>(interval), time_zone);
// case IntervalKind::Millisecond:
// return executeTumble<UInt32, IntervalKind::Millisecond>(*time_column_vec, std::get<1>(interval), time_zone);
case IntervalKind::Second:
return executeTumble<UInt32, IntervalKind::Second>(*time_column_vec, std::get<1>(interval), time_zone);
case IntervalKind::Minute:
@ -183,6 +191,8 @@ struct TimeWindowImpl<TUMBLE>
return executeTumble<UInt16, IntervalKind::Quarter>(*time_column_vec, std::get<1>(interval), time_zone);
case IntervalKind::Year:
return executeTumble<UInt16, IntervalKind::Year>(*time_column_vec, std::get<1>(interval), time_zone);
default:
throw Exception("Fraction seconds are unsupported by windows yet", ErrorCodes::SYNTAX_ERROR);
}
__builtin_unreachable();
}
@ -350,6 +360,16 @@ struct TimeWindowImpl<HOP>
switch (std::get<0>(window_interval))
{
//TODO: add proper support for fractional seconds
// case IntervalKind::Nanosecond:
// return executeHop<UInt32, IntervalKind::Nanosecond>(
// *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
// case IntervalKind::Microsecond:
// return executeHop<UInt32, IntervalKind::Microsecond>(
// *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
// case IntervalKind::Millisecond:
// return executeHop<UInt32, IntervalKind::Millisecond>(
// *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
case IntervalKind::Second:
return executeHop<UInt32, IntervalKind::Second>(
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
@ -374,6 +394,8 @@ struct TimeWindowImpl<HOP>
case IntervalKind::Year:
return executeHop<UInt16, IntervalKind::Year>(
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
default:
throw Exception("Fraction seconds are unsupported by windows yet", ErrorCodes::SYNTAX_ERROR);
}
__builtin_unreachable();
}
@ -487,6 +509,16 @@ struct TimeWindowImpl<WINDOW_ID>
switch (std::get<0>(window_interval))
{
//TODO: add proper support for fractional seconds
// case IntervalKind::Nanosecond:
// return executeHopSlice<UInt32, IntervalKind::Nanosecond>(
// *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
// case IntervalKind::Microsecond:
// return executeHopSlice<UInt32, IntervalKind::Microsecond>(
// *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
// case IntervalKind::Millisecond:
// return executeHopSlice<UInt32, IntervalKind::Millisecond>(
// *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
case IntervalKind::Second:
return executeHopSlice<UInt32, IntervalKind::Second>(
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
@ -511,6 +543,8 @@ struct TimeWindowImpl<WINDOW_ID>
case IntervalKind::Year:
return executeHopSlice<UInt16, IntervalKind::Year>(
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
default:
throw Exception("Fraction seconds are unsupported by windows yet", ErrorCodes::SYNTAX_ERROR);
}
__builtin_unreachable();
}

View File

@ -80,7 +80,32 @@ struct ToStartOfTransform;
TRANSFORM_TIME(Hour)
TRANSFORM_TIME(Minute)
TRANSFORM_TIME(Second)
#undef TRANSFORM_DATE
#undef TRANSFORM_TIME
#define TRANSFORM_SUBSECONDS(INTERVAL_KIND, DEF_SCALE) \
template<> \
struct ToStartOfTransform<IntervalKind::INTERVAL_KIND> \
{ \
static Int64 execute(Int64 t, UInt64 delta, const UInt32 scale) \
{ \
if (scale <= DEF_SCALE) \
{ \
auto val = t * DecimalUtils::scaleMultiplier<DateTime64>(DEF_SCALE - scale); \
if (delta == 1) \
return val; \
else \
return val - (val % delta); \
} \
else \
{ \
return t - (t % (delta * DecimalUtils::scaleMultiplier<DateTime64>(scale - DEF_SCALE))) ; \
} \
} \
};
TRANSFORM_SUBSECONDS(Millisecond, 3)
TRANSFORM_SUBSECONDS(Microsecond, 6)
TRANSFORM_SUBSECONDS(Nanosecond, 9)
#undef TRANSFORM_SUBSECONDS
template <IntervalKind::Kind unit>
struct AddTime;
@ -117,6 +142,25 @@ struct ToStartOfTransform;
ADD_TIME(Second, 1)
#undef ADD_TIME
#define ADD_SUBSECONDS(INTERVAL_KIND, DEF_SCALE) \
template <> \
struct AddTime<IntervalKind::INTERVAL_KIND> \
{ \
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int64 t, UInt64 delta, const UInt32 scale) \
{ \
if (scale < DEF_SCALE) \
{ \
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>(DEF_SCALE - scale); \
} \
else \
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>(scale - DEF_SCALE); \
} \
};
ADD_SUBSECONDS(Millisecond, 3)
ADD_SUBSECONDS(Microsecond, 6)
ADD_SUBSECONDS(Nanosecond, 9)
#undef ADD_SUBSECONDS
template <TimeWindowFunctionName type>
struct TimeWindowImpl
{

View File

@ -0,0 +1,28 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionDateOrDateTimeAddInterval.h>
namespace DB
{
using FunctionSubtractNanoseconds = FunctionDateOrDateTimeAddInterval<SubtractNanosecondsImpl>;
void registerFunctionSubtractNanoseconds(FunctionFactory & factory)
{
factory.registerFunction<FunctionSubtractNanoseconds>();
};
using FunctionSubtractMicroseconds = FunctionDateOrDateTimeAddInterval<SubtractMicrosecondsImpl>;
void registerFunctionSubtractMicroseconds(FunctionFactory & factory)
{
factory.registerFunction<FunctionSubtractMicroseconds>();
};
using FunctionSubtractMilliseconds = FunctionDateOrDateTimeAddInterval<SubtractMillisecondsImpl>;
void registerFunctionSubtractMilliseconds(FunctionFactory & factory)
{
factory.registerFunction<FunctionSubtractMilliseconds>();
};
}

View File

@ -13,7 +13,7 @@ namespace DB
* * DateTime64 value and scale factor (2)
* * DateTime64 broken down to components, result of execute is then re-assembled back into DateTime64 value (3)
*
* Suitable Transfotm-types are commonly used in Date/DateTime manipulation functions,
* Suitable Transform-types are commonly used in Date/DateTime manipulation functions,
* and should implement static (or const) function with following signatures:
* 1:
* R execute(Int64 whole_value, ... )

View File

@ -0,0 +1,28 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionDateOrDateTimeAddInterval.h>
namespace DB
{
using FunctionAddNanoseconds = FunctionDateOrDateTimeAddInterval<AddNanosecondsImpl>;
void registerFunctionAddNanoseconds(FunctionFactory & factory)
{
factory.registerFunction<FunctionAddNanoseconds>();
};
using FunctionAddMicroseconds = FunctionDateOrDateTimeAddInterval<AddMicrosecondsImpl>;
void registerFunctionAddMicroseconds(FunctionFactory & factory)
{
factory.registerFunction<FunctionAddMicroseconds>();
};
using FunctionAddMilliseconds = FunctionDateOrDateTimeAddInterval<AddMillisecondsImpl>;
void registerFunctionAddMilliseconds(FunctionFactory & factory)
{
factory.registerFunction<FunctionAddMilliseconds>();
};
}

View File

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

View File

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

View File

@ -11,6 +11,9 @@ void registerFunctionToDayOfWeek(FunctionFactory &);
void registerFunctionToDayOfYear(FunctionFactory &);
void registerFunctionToHour(FunctionFactory &);
void registerFunctionToMinute(FunctionFactory &);
void registerFunctionToStartOfNanosecond(FunctionFactory &);
void registerFunctionToStartOfMicrosecond(FunctionFactory &);
void registerFunctionToStartOfMillisecond(FunctionFactory &);
void registerFunctionToStartOfSecond(FunctionFactory &);
void registerFunctionToSecond(FunctionFactory &);
void registerFunctionToStartOfDay(FunctionFactory &);
@ -47,6 +50,9 @@ void registerFunctionTimeSlots(FunctionFactory &);
void registerFunctionToYYYYMM(FunctionFactory &);
void registerFunctionToYYYYMMDD(FunctionFactory &);
void registerFunctionToYYYYMMDDhhmmss(FunctionFactory &);
void registerFunctionAddNanoseconds(FunctionFactory &);
void registerFunctionAddMicroseconds(FunctionFactory &);
void registerFunctionAddMilliseconds(FunctionFactory &);
void registerFunctionAddSeconds(FunctionFactory &);
void registerFunctionAddMinutes(FunctionFactory &);
void registerFunctionAddHours(FunctionFactory &);
@ -55,6 +61,9 @@ void registerFunctionAddWeeks(FunctionFactory &);
void registerFunctionAddMonths(FunctionFactory &);
void registerFunctionAddQuarters(FunctionFactory &);
void registerFunctionAddYears(FunctionFactory &);
void registerFunctionSubtractNanoseconds(FunctionFactory &);
void registerFunctionSubtractMicroseconds(FunctionFactory &);
void registerFunctionSubtractMilliseconds(FunctionFactory &);
void registerFunctionSubtractSeconds(FunctionFactory &);
void registerFunctionSubtractMinutes(FunctionFactory &);
void registerFunctionSubtractHours(FunctionFactory &);
@ -93,6 +102,9 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToStartOfMonth(factory);
registerFunctionToStartOfQuarter(factory);
registerFunctionToStartOfYear(factory);
registerFunctionToStartOfNanosecond(factory);
registerFunctionToStartOfMicrosecond(factory);
registerFunctionToStartOfMillisecond(factory);
registerFunctionToStartOfSecond(factory);
registerFunctionToStartOfMinute(factory);
registerFunctionToStartOfFiveMinute(factory);
@ -119,6 +131,9 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToYYYYMM(factory);
registerFunctionToYYYYMMDD(factory);
registerFunctionToYYYYMMDDhhmmss(factory);
registerFunctionAddNanoseconds(factory);
registerFunctionAddMicroseconds(factory);
registerFunctionAddMilliseconds(factory);
registerFunctionAddSeconds(factory);
registerFunctionAddMinutes(factory);
registerFunctionAddHours(factory);
@ -127,6 +142,9 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionAddMonths(factory);
registerFunctionAddQuarters(factory);
registerFunctionAddYears(factory);
registerFunctionSubtractNanoseconds(factory);
registerFunctionSubtractMicroseconds(factory);
registerFunctionSubtractMilliseconds(factory);
registerFunctionSubtractSeconds(factory);
registerFunctionSubtractMinutes(factory);
registerFunctionSubtractHours(factory);

View File

@ -33,184 +33,273 @@ namespace
template <>
struct Transform<IntervalKind::Year>
{
static constexpr auto name = function_name;
static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone)
static UInt16 execute(UInt16 d, Int64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(DayNum(d), years);
}
static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone)
static UInt16 execute(Int32 d, Int64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
}
static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone)
static UInt16 execute(UInt32 t, Int64 years, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
}
static UInt16 execute(Int64 t, UInt64 years, const DateLUTImpl & time_zone)
static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years);
}
};
template <>
struct Transform<IntervalKind::Quarter>
{
static constexpr auto name = function_name;
static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone)
static UInt16 execute(UInt16 d, Int64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(DayNum(d), quarters);
}
static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone)
static UInt16 execute(Int32 d, Int64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
}
static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone)
static UInt16 execute(UInt32 t, Int64 quarters, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
}
static UInt16 execute(Int64 t, UInt64 quarters, const DateLUTImpl & time_zone)
static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters);
}
};
template <>
struct Transform<IntervalKind::Month>
{
static constexpr auto name = function_name;
static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone)
static UInt16 execute(UInt16 d, Int64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(DayNum(d), months);
}
static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone)
static UInt16 execute(Int32 d, Int64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
}
static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone)
static UInt16 execute(UInt32 t, Int64 months, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
}
static UInt16 execute(Int64 t, UInt64 months, const DateLUTImpl & time_zone)
static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months);
}
};
template <>
struct Transform<IntervalKind::Week>
{
static constexpr auto name = function_name;
static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone)
static UInt16 execute(UInt16 d, Int64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(DayNum(d), weeks);
}
static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone)
static UInt16 execute(Int32 d, Int64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
}
static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone)
static UInt16 execute(UInt32 t, Int64 weeks, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
}
static UInt16 execute(Int64 t, UInt64 weeks, const DateLUTImpl & time_zone)
static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks);
}
};
template <>
struct Transform<IntervalKind::Day>
{
static constexpr auto name = function_name;
static UInt32 execute(UInt16 d, UInt64 days, const DateLUTImpl & time_zone)
static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days);
}
static UInt32 execute(Int32 d, UInt64 days, const DateLUTImpl & time_zone)
static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days);
}
static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone)
static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days);
}
static UInt32 execute(Int64 t, UInt64 days, const DateLUTImpl & time_zone)
static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days);
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days);
}
};
template <>
struct Transform<IntervalKind::Hour>
{
static constexpr auto name = function_name;
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
static UInt32 execute(Int64 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfHourInterval(t, hours);
}
static UInt32 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfHourInterval(t / scale_multiplier, hours);
}
};
template <>
struct Transform<IntervalKind::Minute>
{
static constexpr auto name = function_name;
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone)
static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfMinuteInterval(t, minutes);
}
static UInt32 execute(Int64 t, UInt64 minutes, const DateLUTImpl & time_zone)
static UInt32 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMinuteInterval(t, minutes);
return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes);
}
};
template <>
struct Transform<IntervalKind::Second>
{
static constexpr auto name = function_name;
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone)
static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64)
{
return time_zone.toStartOfSecondInterval(t, seconds);
}
static Int64 execute(Int64 t, UInt64 seconds, const DateLUTImpl & time_zone)
static UInt32 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfSecondInterval(t, seconds);
return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds);
}
};
template <>
struct Transform<IntervalKind::Millisecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { return dateTimeIsNotSupported(function_name); }
static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier)
{
if (scale_multiplier < 1000)
{
Int64 t_milliseconds = t * (static_cast<Int64>(1000) / scale_multiplier);
if (likely(t >= 0))
return t_milliseconds / milliseconds * milliseconds;
else
return ((t_milliseconds + 1) / milliseconds - 1) * milliseconds;
}
else if (scale_multiplier > 1000)
{
Int64 scale_diff = scale_multiplier / static_cast<Int64>(1000);
if (likely(t >= 0))
return t / milliseconds / scale_diff * milliseconds;
else
return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds;
}
else
if (likely(t >= 0))
return t / milliseconds * milliseconds;
else
return ((t + 1) / milliseconds - 1) * milliseconds;
}
};
template <>
struct Transform<IntervalKind::Microsecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { return dateTimeIsNotSupported(function_name); }
static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier)
{
if (scale_multiplier < 1000000)
{
Int64 t_microseconds = t * (static_cast<Int64>(1000000) / scale_multiplier);
if (likely(t >= 0))
return t_microseconds / microseconds * microseconds;
else
return ((t_microseconds + 1) / microseconds - 1) * microseconds;
}
else if (scale_multiplier > 1000000)
{
Int64 scale_diff = scale_multiplier / static_cast<Int64>(1000000);
if (likely(t >= 0))
return t / microseconds / scale_diff * microseconds;
else
return ((t + 1) / microseconds / scale_diff - 1) * microseconds;
}
else
if (likely(t >= 0))
return t / microseconds * microseconds;
else
return ((t + 1) / microseconds - 1) * microseconds;
}
};
template <>
struct Transform<IntervalKind::Nanosecond>
{
static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { return dateTimeIsNotSupported(function_name); }
static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier)
{
if (scale_multiplier < 1000000000)
{
Int64 t_nanoseconds = t * (static_cast<Int64>(1000000000) / scale_multiplier);
if (likely(t >= 0))
return t_nanoseconds / nanoseconds * nanoseconds;
else
return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds;
}
else
if (likely(t >= 0))
return t / nanoseconds * nanoseconds;
else
return ((t + 1) / nanoseconds - 1) * nanoseconds;
}
};
class FunctionToStartOfInterval : public IFunction
{
@ -240,6 +329,7 @@ public:
const DataTypeInterval * interval_type = nullptr;
bool result_type_is_date = false;
bool result_type_is_datetime = false;
auto check_interval_argument = [&]
{
interval_type = checkAndGetDataType<DataTypeInterval>(arguments[1].type.get());
@ -251,6 +341,8 @@ public:
result_type_is_date = (interval_type->getKind() == IntervalKind::Year)
|| (interval_type->getKind() == IntervalKind::Quarter) || (interval_type->getKind() == IntervalKind::Month)
|| (interval_type->getKind() == IntervalKind::Week);
result_type_is_datetime = (interval_type->getKind() == IntervalKind::Day) || (interval_type->getKind() == IntervalKind::Hour)
|| (interval_type->getKind() == IntervalKind::Minute) || (interval_type->getKind() == IntervalKind::Second);
};
auto check_timezone_argument = [&]
@ -263,7 +355,7 @@ public:
if (first_argument_is_date && result_type_is_date)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The timezone argument of function {} with interval type {} is allowed only when the 1st argument "
"has the type DateTime",
"has the type DateTime or DateTime64",
getName(), interval_type->getKind().toString());
};
@ -288,19 +380,33 @@ public:
if (result_type_is_date)
return std::make_shared<DataTypeDate>();
else
else if (result_type_is_datetime)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
else
{
auto scale = 0;
if (interval_type->getKind() == IntervalKind::Nanosecond)
scale = 9;
else if (interval_type->getKind() == IntervalKind::Microsecond)
scale = 6;
else if (interval_type->getKind() == IntervalKind::Millisecond)
scale = 3;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /* input_rows_count */) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override
{
const auto & time_column = arguments[0];
const auto & interval_column = arguments[1];
const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0);
auto result_column = dispatchForColumns(time_column, interval_column, time_zone);
auto result_column = dispatchForColumns(time_column, interval_column, result_type, time_zone);
return result_column;
}
@ -316,33 +422,36 @@ public:
private:
ColumnPtr dispatchForColumns(
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone) const
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const
{
const auto & from_datatype = *time_column.type.get();
const auto which_type = WhichDataType(from_datatype);
if (which_type.isDateTime64())
{
const auto * time_column_vec = checkAndGetColumn<DataTypeDateTime64::ColumnType>(time_column.column.get());
auto scale = assert_cast<const DataTypeDateTime64 &>(from_datatype).getScale();
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale);
}
if (which_type.isDateTime())
{
const auto * time_column_vec = checkAndGetColumn<ColumnUInt32>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime&>(from_datatype), *time_column_vec, interval_column, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
}
if (which_type.isDate())
{
const auto * time_column_vec = checkAndGetColumn<ColumnUInt16>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate&>(from_datatype), *time_column_vec, interval_column, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDate&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
}
if (which_type.isDate32())
{
const auto * time_column_vec = checkAndGetColumn<ColumnInt32>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32&>(from_datatype), *time_column_vec, interval_column, time_zone);
}
if (which_type.isDateTime64())
{
const auto * time_column_vec = checkAndGetColumn<DataTypeDateTime64::ColumnType>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64&>(from_datatype), *time_column_vec, interval_column, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32&>(from_datatype), *time_column_vec, interval_column, result_type, time_zone);
}
throw Exception(
"Illegal column for first argument of function " + getName() + ". Must contain dates or dates with time",
@ -351,7 +460,8 @@ private:
template <typename ColumnType, typename FromDataType>
ColumnPtr dispatchForIntervalColumn(
const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone) const
const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const
{
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
if (!interval_type)
@ -368,49 +478,52 @@ private:
switch (interval_type->getKind())
{
case IntervalKind::Nanosecond:
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Nanosecond>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Microsecond:
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Microsecond>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Millisecond:
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Millisecond>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Second:
return execute<FromDataType, UInt32, IntervalKind::Second>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDateTime, IntervalKind::Second>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Minute:
return execute<FromDataType, UInt32, IntervalKind::Minute>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDateTime, IntervalKind::Minute>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Hour:
return execute<FromDataType, UInt32, IntervalKind::Hour>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDateTime, IntervalKind::Hour>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Day:
return execute<FromDataType, UInt32, IntervalKind::Day>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDateTime, IntervalKind::Day>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Week:
return execute<FromDataType, UInt16, IntervalKind::Week>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDate, IntervalKind::Week>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Month:
return execute<FromDataType, UInt16, IntervalKind::Month>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDate, IntervalKind::Month>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Quarter:
return execute<FromDataType, UInt16, IntervalKind::Quarter>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDate, IntervalKind::Quarter>(from, time_column, num_units, result_type, time_zone, scale);
case IntervalKind::Year:
return execute<FromDataType, UInt16, IntervalKind::Year>(from, time_column, num_units, time_zone);
return execute<FromDataType, DataTypeDate, IntervalKind::Year>(from, time_column, num_units, result_type, time_zone, scale);
}
__builtin_unreachable();
}
template <typename FromDataType, typename ToType, IntervalKind::Kind unit, typename ColumnType>
ColumnPtr execute(const FromDataType & from_datatype, const ColumnType & time_column, UInt64 num_units, const DateLUTImpl & time_zone) const
template <typename FromDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>
ColumnPtr execute(const FromDataType &, const ColumnType & time_column_type, Int64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const
{
const auto & time_data = time_column.getData();
size_t size = time_column.size();
auto result = ColumnVector<ToType>::create();
auto & result_data = result->getData();
using ToColumnType = typename ToDataType::ColumnType;
const auto & time_data = time_column_type.getData();
size_t size = time_data.size();
auto result_col = result_type->createColumn();
auto *col_to = assert_cast<ToColumnType *>(result_col.get());
auto & result_data = col_to->getData();
result_data.resize(size);
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
{
const auto transform = TransformDateTime64<Transform<unit>>{from_datatype.getScale()};
for (size_t i = 0; i != size; ++i)
result_data[i] = transform.execute(time_data[i], num_units, time_zone);
}
else
{
for (size_t i = 0; i != size; ++i)
result_data[i] = Transform<unit>::execute(time_data[i], num_units, time_zone);
}
return result;
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
for (size_t i = 0; i != size; ++i)
result_data[i] = Transform<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier);
return result_col;
}
};

View File

@ -0,0 +1,30 @@
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
namespace DB
{
using FunctionToStartOfMillisecond = FunctionDateOrDateTimeToSomething<DataTypeDateTime64, ToStartOfMillisecondImpl>;
void registerFunctionToStartOfMillisecond(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfMillisecond>();
}
using FunctionToStartOfMicrosecond = FunctionDateOrDateTimeToSomething<DataTypeDateTime64, ToStartOfMicrosecondImpl>;
void registerFunctionToStartOfMicrosecond(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfMicrosecond>();
}
using FunctionToStartOfNanosecond = FunctionDateOrDateTimeToSomething<DataTypeDateTime64, ToStartOfNanosecondImpl>;
void registerFunctionToStartOfNanosecond(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfNanosecond>();
}
}

View File

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

View File

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

View File

@ -26,10 +26,10 @@ extern const int CANNOT_LOAD_CATBOOST_MODEL;
extern const int CANNOT_APPLY_CATBOOST_MODEL;
}
/// CatBoost wrapper interface functions.
struct CatBoostWrapperAPI
class CatBoostWrapperAPI
{
public:
using ModelCalcerHandle = void;
ModelCalcerHandle * (* ModelCalcerCreate)(); // NOLINT
@ -68,9 +68,6 @@ struct CatBoostWrapperAPI
};
namespace
{
class CatBoostModelHolder
{
private:
@ -84,7 +81,61 @@ public:
};
class CatBoostModelImpl : public ICatBoostModel
/// Holds CatBoost wrapper library and provides wrapper interface.
class CatBoostLibHolder
{
public:
explicit CatBoostLibHolder(std::string lib_path_) : lib_path(std::move(lib_path_)), lib(lib_path) { initAPI(); }
const CatBoostWrapperAPI & getAPI() const { return api; }
const std::string & getCurrentPath() const { return lib_path; }
private:
CatBoostWrapperAPI api;
std::string lib_path;
SharedLibrary lib;
void initAPI()
{
load(api.ModelCalcerCreate, "ModelCalcerCreate");
load(api.ModelCalcerDelete, "ModelCalcerDelete");
load(api.GetErrorString, "GetErrorString");
load(api.LoadFullModelFromFile, "LoadFullModelFromFile");
load(api.CalcModelPredictionFlat, "CalcModelPredictionFlat");
load(api.CalcModelPrediction, "CalcModelPrediction");
load(api.CalcModelPredictionWithHashedCatFeatures, "CalcModelPredictionWithHashedCatFeatures");
load(api.GetStringCatFeatureHash, "GetStringCatFeatureHash");
load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash");
load(api.GetFloatFeaturesCount, "GetFloatFeaturesCount");
load(api.GetCatFeaturesCount, "GetCatFeaturesCount");
tryLoad(api.CheckModelMetadataHasKey, "CheckModelMetadataHasKey");
tryLoad(api.GetModelInfoValueSize, "GetModelInfoValueSize");
tryLoad(api.GetModelInfoValue, "GetModelInfoValue");
tryLoad(api.GetTreeCount, "GetTreeCount");
tryLoad(api.GetDimensionsCount, "GetDimensionsCount");
}
template <typename T>
void load(T& func, const std::string & name) { func = lib.get<T>(name); }
template <typename T>
void tryLoad(T& func, const std::string & name) { func = lib.tryGet<T>(name); }
};
std::shared_ptr<CatBoostLibHolder> getCatBoostWrapperHolder(const std::string & lib_path)
{
static std::shared_ptr<CatBoostLibHolder> ptr;
static std::mutex mutex;
std::lock_guard lock(mutex);
if (!ptr || ptr->getCurrentPath() != lib_path)
ptr = std::make_shared<CatBoostLibHolder>(lib_path);
return ptr;
}
class CatBoostModelImpl
{
public:
CatBoostModelImpl(const CatBoostWrapperAPI * api_, const std::string & model_path) : api(api_)
@ -92,13 +143,15 @@ public:
handle = std::make_unique<CatBoostModelHolder>(api);
if (!handle)
{
std::string msg = "Cannot create CatBoost model: ";
throw Exception(msg + api->GetErrorString(), ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL);
throw Exception(ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL,
"Cannot create CatBoost model: {}",
api->GetErrorString());
}
if (!api->LoadFullModelFromFile(handle->get(), model_path.c_str()))
{
std::string msg = "Cannot load CatBoost model: ";
throw Exception(msg + api->GetErrorString(), ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL);
throw Exception(ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL,
"Cannot load CatBoost model: {}",
api->GetErrorString());
}
float_features_count = api->GetFloatFeaturesCount(handle->get());
@ -108,32 +161,22 @@ public:
tree_count = api->GetDimensionsCount(handle->get());
}
ColumnPtr evaluate(const ColumnRawPtrs & columns) const override
ColumnPtr evaluate(const ColumnRawPtrs & columns) const
{
if (columns.empty())
throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got empty columns list for CatBoost model.");
if (columns.size() != float_features_count + cat_features_count)
{
std::string msg;
{
WriteBufferFromString buffer(msg);
buffer << "Number of columns is different with number of features: ";
buffer << columns.size() << " vs " << float_features_count << " + " << cat_features_count;
}
throw Exception(msg, ErrorCodes::BAD_ARGUMENTS);
}
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Number of columns is different with number of features: columns size {} float features size {} + cat features size {}",
float_features_count,
cat_features_count);
for (size_t i = 0; i < float_features_count; ++i)
{
if (!columns[i]->isNumeric())
{
std::string msg;
{
WriteBufferFromString buffer(msg);
buffer << "Column " << i << " should be numeric to make float feature.";
}
throw Exception(msg, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column {} should be numeric to make float feature.", i);
}
}
@ -142,16 +185,13 @@ public:
{
const auto * column = columns[i];
if (column->isNumeric())
{
cat_features_are_strings = false;
}
else if (!(typeid_cast<const ColumnString *>(column)
|| typeid_cast<const ColumnFixedString *>(column)))
{
std::string msg;
{
WriteBufferFromString buffer(msg);
buffer << "Column " << i << " should be numeric or string.";
}
throw Exception(msg, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column {} should be numeric or string.", i);
}
}
@ -187,9 +227,9 @@ public:
return ColumnTuple::create(std::move(mutable_columns));
}
size_t getFloatFeaturesCount() const override { return float_features_count; }
size_t getCatFeaturesCount() const override { return cat_features_count; }
size_t getTreeCount() const override { return tree_count; }
size_t getFloatFeaturesCount() const { return float_features_count; }
size_t getCatFeaturesCount() const { return cat_features_count; }
size_t getTreeCount() const { return tree_count; }
private:
std::unique_ptr<CatBoostModelHolder> handle;
@ -435,66 +475,6 @@ private:
}
};
/// Holds CatBoost wrapper library and provides wrapper interface.
class CatBoostLibHolder: public CatBoostWrapperAPIProvider
{
public:
explicit CatBoostLibHolder(std::string lib_path_) : lib_path(std::move(lib_path_)), lib(lib_path) { initAPI(); }
const CatBoostWrapperAPI & getAPI() const override { return api; }
const std::string & getCurrentPath() const { return lib_path; }
private:
CatBoostWrapperAPI api;
std::string lib_path;
SharedLibrary lib;
void initAPI();
template <typename T>
void load(T& func, const std::string & name) { func = lib.get<T>(name); }
template <typename T>
void tryLoad(T& func, const std::string & name) { func = lib.tryGet<T>(name); }
};
void CatBoostLibHolder::initAPI()
{
load(api.ModelCalcerCreate, "ModelCalcerCreate");
load(api.ModelCalcerDelete, "ModelCalcerDelete");
load(api.GetErrorString, "GetErrorString");
load(api.LoadFullModelFromFile, "LoadFullModelFromFile");
load(api.CalcModelPredictionFlat, "CalcModelPredictionFlat");
load(api.CalcModelPrediction, "CalcModelPrediction");
load(api.CalcModelPredictionWithHashedCatFeatures, "CalcModelPredictionWithHashedCatFeatures");
load(api.GetStringCatFeatureHash, "GetStringCatFeatureHash");
load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash");
load(api.GetFloatFeaturesCount, "GetFloatFeaturesCount");
load(api.GetCatFeaturesCount, "GetCatFeaturesCount");
tryLoad(api.CheckModelMetadataHasKey, "CheckModelMetadataHasKey");
tryLoad(api.GetModelInfoValueSize, "GetModelInfoValueSize");
tryLoad(api.GetModelInfoValue, "GetModelInfoValue");
tryLoad(api.GetTreeCount, "GetTreeCount");
tryLoad(api.GetDimensionsCount, "GetDimensionsCount");
}
std::shared_ptr<CatBoostLibHolder> getCatBoostWrapperHolder(const std::string & lib_path)
{
static std::shared_ptr<CatBoostLibHolder> ptr;
static std::mutex mutex;
std::lock_guard lock(mutex);
if (!ptr || ptr->getCurrentPath() != lib_path)
ptr = std::make_shared<CatBoostLibHolder>(lib_path);
return ptr;
}
}
CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::string lib_path_,
const ExternalLoadableLifetime & lifetime_)
: name(std::move(name_)), model_path(std::move(model_path_)), lib_path(std::move(lib_path_)), lifetime(lifetime_)
@ -502,43 +482,28 @@ CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::st
api_provider = getCatBoostWrapperHolder(lib_path);
api = &api_provider->getAPI();
model = std::make_unique<CatBoostModelImpl>(api, model_path);
float_features_count = model->getFloatFeaturesCount();
cat_features_count = model->getCatFeaturesCount();
tree_count = model->getTreeCount();
}
const ExternalLoadableLifetime & CatBoostModel::getLifetime() const
{
return lifetime;
}
bool CatBoostModel::isModified() const
{
return true;
}
std::shared_ptr<const IExternalLoadable> CatBoostModel::clone() const
{
return std::make_shared<CatBoostModel>(name, model_path, lib_path, lifetime);
}
CatBoostModel::~CatBoostModel() = default;
size_t CatBoostModel::getFloatFeaturesCount() const
{
return float_features_count;
return model->getFloatFeaturesCount();
}
size_t CatBoostModel::getCatFeaturesCount() const
{
return cat_features_count;
return model->getCatFeaturesCount();
}
size_t CatBoostModel::getTreeCount() const
{
return tree_count;
return model->getTreeCount();
}
DataTypePtr CatBoostModel::getReturnType() const
{
size_t tree_count = getTreeCount();
auto type = std::make_shared<DataTypeFloat64>();
if (tree_count == 1)
return type;
@ -552,6 +517,7 @@ ColumnPtr CatBoostModel::evaluate(const ColumnRawPtrs & columns) const
{
if (!model)
throw Exception("CatBoost model was not loaded.", ErrorCodes::LOGICAL_ERROR);
return model->evaluate(columns);
}

View File

@ -8,47 +8,32 @@
namespace DB
{
/// CatBoost wrapper interface functions.
struct CatBoostWrapperAPI;
class CatBoostWrapperAPIProvider
{
public:
virtual ~CatBoostWrapperAPIProvider() = default;
virtual const CatBoostWrapperAPI & getAPI() const = 0;
};
/// CatBoost model interface.
class ICatBoostModel
{
public:
virtual ~ICatBoostModel() = default;
/// Evaluate model. Use first `float_features_count` columns as float features,
/// the others `cat_features_count` as categorical features.
virtual ColumnPtr evaluate(const ColumnRawPtrs & columns) const = 0;
virtual size_t getFloatFeaturesCount() const = 0;
virtual size_t getCatFeaturesCount() const = 0;
virtual size_t getTreeCount() const = 0;
};
class CatBoostLibHolder;
class CatBoostWrapperAPI;
class CatBoostModelImpl;
class IDataType;
using DataTypePtr = std::shared_ptr<const IDataType>;
/// General ML model evaluator interface.
class IModel : public IExternalLoadable
class IMLModel : public IExternalLoadable
{
public:
IMLModel() = default;
virtual ColumnPtr evaluate(const ColumnRawPtrs & columns) const = 0;
virtual std::string getTypeName() const = 0;
virtual DataTypePtr getReturnType() const = 0;
virtual ~IMLModel() override = default;
};
class CatBoostModel : public IModel
class CatBoostModel : public IMLModel
{
public:
CatBoostModel(std::string name, std::string model_path,
std::string lib_path, const ExternalLoadableLifetime & lifetime);
~CatBoostModel() override;
ColumnPtr evaluate(const ColumnRawPtrs & columns) const override;
std::string getTypeName() const override { return "catboost"; }
@ -59,29 +44,28 @@ public:
/// IExternalLoadable interface.
const ExternalLoadableLifetime & getLifetime() const override;
const ExternalLoadableLifetime & getLifetime() const override { return lifetime; }
std::string getLoadableName() const override { return name; }
bool supportUpdates() const override { return true; }
bool isModified() const override;
bool isModified() const override { return true; }
std::shared_ptr<const IExternalLoadable> clone() const override;
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<CatBoostModel>(name, model_path, lib_path, lifetime);
}
private:
const std::string name;
std::string model_path;
std::string lib_path;
ExternalLoadableLifetime lifetime;
std::shared_ptr<CatBoostWrapperAPIProvider> api_provider;
std::shared_ptr<CatBoostLibHolder> api_provider;
const CatBoostWrapperAPI * api;
std::unique_ptr<ICatBoostModel> model;
size_t float_features_count;
size_t cat_features_count;
size_t tree_count;
std::unique_ptr<CatBoostModelImpl> model;
void init();
};

View File

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

View File

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

View File

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

View File

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

View File

@ -15,14 +15,14 @@ namespace DB
class ExternalModelsLoader : public ExternalLoader, WithContext
{
public:
using ModelPtr = std::shared_ptr<const IModel>;
using ModelPtr = std::shared_ptr<const IMLModel>;
/// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
explicit ExternalModelsLoader(ContextPtr context_);
ModelPtr getModel(const std::string & model_name) const
{
return std::static_pointer_cast<const IModel>(load(model_name));
return std::static_pointer_cast<const IMLModel>(load(model_name));
}
void reloadModel(const std::string & model_name) const

View File

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

View File

@ -1237,16 +1237,16 @@ NO_INLINE IColumn::Filter joinRightColumns(
{
const IColumn & left_asof_key = added_columns.leftAsofKey();
auto [block, row_num] = mapped->findAsof(left_asof_key, i);
if (block)
auto row_ref = mapped->findAsof(left_asof_key, i);
if (row_ref.block)
{
setUsed<need_filter>(filter, i);
if constexpr (multiple_disjuncts)
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(block, row_num, 0);
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(row_ref.block, row_ref.row_num, 0);
else
used_flags.template setUsed<jf.need_flags, multiple_disjuncts>(find_result);
added_columns.appendFromBlock<jf.add_missing>(*block, row_num);
added_columns.appendFromBlock<jf.add_missing>(*row_ref.block, row_ref.row_num);
}
else
addNotFoundRow<jf.add_missing, jf.need_replication>(added_columns, current_offset);

View File

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

View File

@ -1,5 +1,6 @@
#include <Interpreters/RowRefs.h>
#include <Common/RadixSort.h>
#include <AggregateFunctions/Helpers.h>
#include <Columns/IColumn.h>
#include <DataTypes/IDataType.h>
@ -44,38 +45,52 @@ class SortedLookupVector : public SortedLookupVectorBase
{
struct Entry
{
/// We don't store a RowRef and instead keep it's members separately (and return a tuple) to reduce the memory usage.
/// For example, for sizeof(T) == 4 => sizeof(Entry) == 16 (while before it would be 20). Then when you put it into a vector, the effect is even greater
decltype(RowRef::block) block;
decltype(RowRef::row_num) row_num;
TKey asof_value;
TKey value;
uint32_t row_ref_index;
Entry() = delete;
Entry(TKey v, const Block * b, size_t r) : block(b), row_num(r), asof_value(v) { }
Entry(TKey value_, uint32_t row_ref_index_)
: value(value_)
, row_ref_index(row_ref_index_)
{ }
bool operator<(const Entry & other) const { return asof_value < other.asof_value; }
};
struct LessEntryOperator
{
ALWAYS_INLINE bool operator()(const Entry & lhs, const Entry & rhs) const
{
return lhs.value < rhs.value;
}
};
struct GreaterEntryOperator
{
bool operator()(Entry const & a, Entry const & b) const { return a.asof_value > b.asof_value; }
ALWAYS_INLINE bool operator()(const Entry & lhs, const Entry & rhs) const
{
return lhs.value > rhs.value;
}
};
public:
using Base = std::vector<Entry>;
using Keys = std::vector<TKey>;
static constexpr bool isDescending = (inequality == ASOF::Inequality::Greater || inequality == ASOF::Inequality::GreaterOrEquals);
static constexpr bool isStrict = (inequality == ASOF::Inequality::Less) || (inequality == ASOF::Inequality::Greater);
using Entries = PaddedPODArray<Entry>;
using RowRefs = PaddedPODArray<RowRef>;
static constexpr bool is_descending = (inequality == ASOF::Inequality::Greater || inequality == ASOF::Inequality::GreaterOrEquals);
static constexpr bool is_strict = (inequality == ASOF::Inequality::Less) || (inequality == ASOF::Inequality::Greater);
void insert(const IColumn & asof_column, const Block * block, size_t row_num) override
{
using ColumnType = ColumnVectorOrDecimal<TKey>;
const auto & column = assert_cast<const ColumnType &>(asof_column);
TKey k = column.getElement(row_num);
TKey key = column.getElement(row_num);
assert(!sorted.load(std::memory_order_acquire));
array.emplace_back(k, block, row_num);
entries.emplace_back(key, row_refs.size());
row_refs.emplace_back(RowRef(block, row_num));
}
/// Unrolled version of upper_bound and lower_bound
@ -84,30 +99,30 @@ public:
/// at https://en.algorithmica.org/hpc/data-structures/s-tree/
size_t boundSearch(TKey value)
{
size_t size = array.size();
size_t size = entries.size();
size_t low = 0;
/// This is a single binary search iteration as a macro to unroll. Takes into account the inequality:
/// isStrict -> Equal values are not requested
/// isDescending -> The vector is sorted in reverse (for greater or greaterOrEquals)
/// is_strict -> Equal values are not requested
/// is_descending -> The vector is sorted in reverse (for greater or greaterOrEquals)
#define BOUND_ITERATION \
{ \
size_t half = size / 2; \
size_t other_half = size - half; \
size_t probe = low + half; \
size_t other_low = low + other_half; \
TKey v = array[probe].asof_value; \
TKey & v = entries[probe].value; \
size = half; \
if constexpr (isDescending) \
if constexpr (is_descending) \
{ \
if constexpr (isStrict) \
if constexpr (is_strict) \
low = value <= v ? other_low : low; \
else \
low = value < v ? other_low : low; \
} \
else \
{ \
if constexpr (isStrict) \
if constexpr (is_strict) \
low = value >= v ? other_low : low; \
else \
low = value > v ? other_low : low; \
@ -130,7 +145,7 @@ public:
return low;
}
std::tuple<decltype(RowRef::block), decltype(RowRef::row_num)> findAsof(const IColumn & asof_column, size_t row_num) override
RowRef findAsof(const IColumn & asof_column, size_t row_num) override
{
sort();
@ -139,8 +154,11 @@ public:
TKey k = column.getElement(row_num);
size_t pos = boundSearch(k);
if (pos != array.size())
return std::make_tuple(array[pos].block, array[pos].row_num);
if (pos != entries.size())
{
size_t row_ref_index = entries[pos].row_ref_index;
return row_refs[row_ref_index];
}
return {nullptr, 0};
}
@ -148,7 +166,8 @@ public:
private:
std::atomic<bool> sorted = false;
mutable std::mutex lock;
Base array;
Entries entries;
RowRefs row_refs;
// Double checked locking with SC atomics works in C++
// https://preshing.com/20130930/double-checked-locking-is-fixed-in-cpp11/
@ -160,12 +179,37 @@ private:
if (!sorted.load(std::memory_order_acquire))
{
std::lock_guard<std::mutex> l(lock);
if (!sorted.load(std::memory_order_relaxed))
{
if constexpr (isDescending)
::sort(array.begin(), array.end(), GreaterEntryOperator());
if constexpr (std::is_arithmetic_v<TKey> && !std::is_floating_point_v<TKey>)
{
if (likely(entries.size() > 256))
{
struct RadixSortTraits : RadixSortNumTraits<TKey>
{
using Element = Entry;
using Result = Element;
static TKey & extractKey(Element & elem) { return elem.value; }
static Element extractResult(Element & elem) { return elem; }
};
if constexpr (is_descending)
RadixSort<RadixSortTraits>::executeLSD(entries.data(), entries.size(), true);
else
RadixSort<RadixSortTraits>::executeLSD(entries.data(), entries.size(), false);
sorted.store(true, std::memory_order_release);
return;
}
}
if constexpr (is_descending)
::sort(entries.begin(), entries.end(), GreaterEntryOperator());
else
::sort(array.begin(), array.end());
::sort(entries.begin(), entries.end(), LessEntryOperator());
sorted.store(true, std::memory_order_release);
}
}

View File

@ -146,7 +146,7 @@ private:
struct SortedLookupVectorBase
{
SortedLookupVectorBase() = default;
virtual ~SortedLookupVectorBase() { }
virtual ~SortedLookupVectorBase() = default;
static std::optional<TypeIndex> getTypeSize(const IColumn & asof_column, size_t & type_size);
@ -154,7 +154,7 @@ struct SortedLookupVectorBase
virtual void insert(const IColumn &, const Block *, size_t) = 0;
// This needs to be synchronized internally
virtual std::tuple<decltype(RowRef::block), decltype(RowRef::row_num)> findAsof(const IColumn &, size_t) = 0;
virtual RowRef findAsof(const IColumn &, size_t) = 0;
};

View File

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

View File

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

View File

@ -7,6 +7,27 @@ namespace DB
{
bool parseIntervalKind(IParser::Pos & pos, Expected & expected, IntervalKind & result)
{
if (ParserKeyword("NANOSECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_NANOSECOND").ignore(pos, expected)
|| ParserKeyword("NS").ignore(pos, expected))
{
result = IntervalKind::Nanosecond;
return true;
}
if (ParserKeyword("MICROSECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_MICROSECOND").ignore(pos, expected)
|| ParserKeyword("MCS").ignore(pos, expected))
{
result = IntervalKind::Microsecond;
return true;
}
if (ParserKeyword("MILLISECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_MILLISECOND").ignore(pos, expected)
|| ParserKeyword("MS").ignore(pos, expected))
{
result = IntervalKind::Millisecond;
return true;
}
if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected)
|| ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected))
{

View File

@ -4,7 +4,6 @@
#include <Formats/EscapingRuleUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
@ -242,15 +241,16 @@ std::unordered_map<String, DataTypePtr> TSKVSchemaReader::readRowAndGetNamesAndD
std::unordered_map<String, DataTypePtr> names_and_types;
StringRef name_ref;
String name_tmp;
String name_buf;
String value;
do
{
bool has_value = readName(in, name_ref, name_tmp);
bool has_value = readName(in, name_ref, name_buf);
String name = String(name_ref);
if (has_value)
{
readEscapedString(value, in);
names_and_types[String(name_ref)] = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped);
names_and_types[std::move(name)] = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped);
}
else
{

View File

@ -34,16 +34,16 @@ Block FillingTransform::transformHeader(Block header, const SortDescription & so
template <typename T>
static FillColumnDescription::StepFunction getStepFunction(
IntervalKind kind, Int64 step, const DateLUTImpl & date_lut)
IntervalKind kind, Int64 step, const DateLUTImpl & date_lut, UInt16 scale = DataTypeDateTime64::default_scale)
{
switch (kind)
{
#define DECLARE_CASE(NAME) \
#define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \
return [step, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(get<T>(field), step, date_lut); };
return [step, scale, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(get<T>(field), step, date_lut, scale); };
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
#undef DECLARE_CASE
}
__builtin_unreachable();
}
@ -92,7 +92,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
Int64 avg_seconds = get<Int64>(descr.fill_step) * descr.step_kind->toAvgSeconds();
if (avg_seconds < 86400)
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Value of step is to low ({} seconds). Must be >= 1 day", avg_seconds);
"Value of step is to low ({} seconds). Must be >= 1 day", avg_seconds);
}
if (which.isDate())
@ -108,25 +108,23 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
switch (*descr.step_kind)
{
#define DECLARE_CASE(NAME) \
#define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \
descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \
{ \
auto field_decimal = get<DecimalField<DateTime64>>(field); \
auto components = DecimalUtils::splitWithScaleMultiplier(field_decimal.getValue(), field_decimal.getScaleMultiplier()); \
auto res = Add##NAME##sImpl::execute(components, step, time_zone); \
auto res_decimal = decimalFromComponentsWithMultiplier<DateTime64>(res, field_decimal.getScaleMultiplier()); \
field = DecimalField(res_decimal, field_decimal.getScale()); \
auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, field_decimal.getScale()); \
field = DecimalField(res, field_decimal.getScale()); \
}; \
break;
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
#undef DECLARE_CASE
}
}
else
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"STEP of Interval type can be used only with Date/DateTime types, but got {}", type->getName());
"STEP of Interval type can be used only with Date/DateTime types, but got {}", type->getName());
}
else
{
@ -140,12 +138,12 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
}
FillingTransform::FillingTransform(
const Block & header_, const SortDescription & sort_description_, bool on_totals_)
: ISimpleTransform(header_, transformHeader(header_, sort_description_), true)
, sort_description(sort_description_)
, on_totals(on_totals_)
, filling_row(sort_description_)
, next_row(sort_description_)
const Block & header_, const SortDescription & sort_description_, bool on_totals_)
: ISimpleTransform(header_, transformHeader(header_, sort_description_), true)
, sort_description(sort_description_)
, on_totals(on_totals_)
, filling_row(sort_description_)
, next_row(sort_description_)
{
if (on_totals)
return;
@ -162,14 +160,14 @@ FillingTransform::FillingTransform(
if (!tryConvertFields(descr, type))
throw Exception("Incompatible types of WITH FILL expression values with column type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (type->isValueRepresentedByUnsignedInteger() &&
((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) ||
(!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1))))
(!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1))))
{
throw Exception("WITH FILL bound values cannot be negative for unsigned type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
}
@ -214,7 +212,7 @@ void FillingTransform::transform(Chunk & chunk)
MutableColumns res_other_columns;
auto init_columns_by_positions = [](const Columns & old_columns, Columns & new_columns,
MutableColumns & new_mutable_columns, const Positions & positions)
MutableColumns & new_mutable_columns, const Positions & positions)
{
for (size_t pos : positions)
{

View File

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

View File

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

View File

@ -38,7 +38,7 @@ void StorageSystemModels::fillData(MutableColumns & res_columns, ContextPtr cont
if (load_result.object)
{
const auto model_ptr = std::static_pointer_cast<const IModel>(load_result.object);
const auto model_ptr = std::static_pointer_cast<const IMLModel>(load_result.object);
res_columns[3]->insert(model_ptr->getTypeName());
}
else

View File

@ -57,6 +57,7 @@ namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int SYNTAX_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int INCORRECT_QUERY;
@ -262,7 +263,13 @@ namespace
IntervalKind strToIntervalKind(const String& interval_str)
{
if (interval_str == "Second")
if (interval_str == "Nanosecond")
return IntervalKind::Nanosecond;
else if (interval_str == "Microsecond")
return IntervalKind::Microsecond;
else if (interval_str == "Millisecond")
return IntervalKind::Millisecond;
else if (interval_str == "Second")
return IntervalKind::Second;
else if (interval_str == "Minute")
return IntervalKind::Minute;
@ -307,6 +314,12 @@ namespace
{
switch (kind)
{
case IntervalKind::Nanosecond:
throw Exception("Fractional seconds are not supported by windows yet", ErrorCodes::SYNTAX_ERROR);
case IntervalKind::Microsecond:
throw Exception("Fractional seconds are not supported by windows yet", ErrorCodes::SYNTAX_ERROR);
case IntervalKind::Millisecond:
throw Exception("Fractional seconds are not supported by windows yet", ErrorCodes::SYNTAX_ERROR);
#define CASE_WINDOW_KIND(KIND) \
case IntervalKind::KIND: { \
return AddTime<IntervalKind::KIND>::execute(time_sec, num_units, time_zone); \
@ -738,6 +751,12 @@ UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec)
switch (window_interval_kind)
{
case IntervalKind::Nanosecond:
throw Exception("Fractional seconds are not supported by windows yet", ErrorCodes::SYNTAX_ERROR);
case IntervalKind::Microsecond:
throw Exception("Fractional seconds are not supported by windows yet", ErrorCodes::SYNTAX_ERROR);
case IntervalKind::Millisecond:
throw Exception("Fractional seconds are not supported by windows yet", ErrorCodes::SYNTAX_ERROR);
#define CASE_WINDOW_KIND(KIND) \
case IntervalKind::KIND: \
{ \
@ -773,6 +792,13 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec)
switch (window_interval_kind)
{
case IntervalKind::Nanosecond:
throw Exception("Fractional seconds are not supported by window view yet", ErrorCodes::SYNTAX_ERROR);
case IntervalKind::Microsecond:
throw Exception("Fractional seconds are not supported by window view yet", ErrorCodes::SYNTAX_ERROR);
case IntervalKind::Millisecond:
throw Exception("Fractional seconds are not supported by window view yet", ErrorCodes::SYNTAX_ERROR);
#define CASE_WINDOW_KIND(KIND) \
case IntervalKind::KIND: \
{ \

View File

@ -238,7 +238,7 @@ def _update_dockerfile(repo_path: str, version: ClickHouseVersion):
def update_version_local(repo_path, version, version_type="testing"):
update_contributors()
version.with_description(version_type)
update_cmake_version(version, version_type)
update_cmake_version(version)
_update_changelog(repo_path, version)
_update_dockerfile(repo_path, version)

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -6,9 +6,10 @@ import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node_default', stay_alive=True)
node = cluster.add_instance("node_default", stay_alive=True)
@pytest.fixture(scope='module', autouse=True)
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
@ -18,7 +19,11 @@ def start_cluster():
def test_system_logs_comment():
node.exec_in_container(['bash', '-c', f"""echo "
node.exec_in_container(
[
"bash",
"-c",
f"""echo "
<clickhouse>
<query_log>
<engine>ENGINE = MergeTree
@ -32,11 +37,13 @@ def test_system_logs_comment():
</query_log>
</clickhouse>
" > /etc/clickhouse-server/config.d/yyy-override-query_log.xml
"""])
""",
]
)
node.restart_clickhouse()
node.query("select 1")
node.query("system flush logs")
comment = node.query("SELECT comment FROM system.tables WHERE name = 'query_log'")
assert comment =='test_comment\n'
assert comment == "test_comment\n"

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

Some files were not shown because too many files have changed in this diff Show More