mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Merge branch 'master' into fix_52405
This commit is contained in:
commit
2ee149a100
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -258,9 +258,6 @@
|
|||||||
[submodule "contrib/wyhash"]
|
[submodule "contrib/wyhash"]
|
||||||
path = contrib/wyhash
|
path = contrib/wyhash
|
||||||
url = https://github.com/wangyi-fudan/wyhash
|
url = https://github.com/wangyi-fudan/wyhash
|
||||||
[submodule "contrib/hashidsxx"]
|
|
||||||
path = contrib/hashidsxx
|
|
||||||
url = https://github.com/schoentoon/hashidsxx
|
|
||||||
[submodule "contrib/nats-io"]
|
[submodule "contrib/nats-io"]
|
||||||
path = contrib/nats-io
|
path = contrib/nats-io
|
||||||
url = https://github.com/ClickHouse/nats.c
|
url = https://github.com/ClickHouse/nats.c
|
||||||
|
1
contrib/CMakeLists.txt
vendored
1
contrib/CMakeLists.txt
vendored
@ -164,7 +164,6 @@ add_contrib (libpq-cmake libpq)
|
|||||||
add_contrib (nuraft-cmake NuRaft)
|
add_contrib (nuraft-cmake NuRaft)
|
||||||
add_contrib (fast_float-cmake fast_float)
|
add_contrib (fast_float-cmake fast_float)
|
||||||
add_contrib (datasketches-cpp-cmake datasketches-cpp)
|
add_contrib (datasketches-cpp-cmake datasketches-cpp)
|
||||||
add_contrib (hashidsxx-cmake hashidsxx)
|
|
||||||
|
|
||||||
option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES})
|
option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES})
|
||||||
if (ENABLE_NLP)
|
if (ENABLE_NLP)
|
||||||
|
1
contrib/hashidsxx
vendored
1
contrib/hashidsxx
vendored
@ -1 +0,0 @@
|
|||||||
Subproject commit 783f6911ccfdaca83e3cfac084c4aad888a80cee
|
|
@ -1,14 +0,0 @@
|
|||||||
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/hashidsxx")
|
|
||||||
|
|
||||||
set (SRCS
|
|
||||||
"${LIBRARY_DIR}/hashids.cpp"
|
|
||||||
)
|
|
||||||
|
|
||||||
set (HDRS
|
|
||||||
"${LIBRARY_DIR}/hashids.h"
|
|
||||||
)
|
|
||||||
|
|
||||||
add_library(_hashidsxx ${SRCS} ${HDRS})
|
|
||||||
target_include_directories(_hashidsxx SYSTEM PUBLIC "${LIBRARY_DIR}")
|
|
||||||
|
|
||||||
add_library(ch_contrib::hashidsxx ALIAS _hashidsxx)
|
|
@ -141,7 +141,6 @@ function clone_submodules
|
|||||||
contrib/jemalloc
|
contrib/jemalloc
|
||||||
contrib/replxx
|
contrib/replxx
|
||||||
contrib/wyhash
|
contrib/wyhash
|
||||||
contrib/hashidsxx
|
|
||||||
contrib/c-ares
|
contrib/c-ares
|
||||||
contrib/morton-nd
|
contrib/morton-nd
|
||||||
contrib/xxHash
|
contrib/xxHash
|
||||||
|
@ -1449,7 +1449,7 @@ Using replacement fields, you can define a pattern for the resulting string. “
|
|||||||
| %n | new-line character (‘’) | |
|
| %n | new-line character (‘’) | |
|
||||||
| %p | AM or PM designation | PM |
|
| %p | AM or PM designation | PM |
|
||||||
| %Q | Quarter (1-4) | 1 |
|
| %Q | Quarter (1-4) | 1 |
|
||||||
| %r | 12-hour HH:MM AM/PM time, equivalent to %H:%i %p | 10:30 PM |
|
| %r | 12-hour HH:MM AM/PM time, equivalent to %h:%i %p | 10:30 PM |
|
||||||
| %R | 24-hour HH:MM time, equivalent to %H:%i | 22:33 |
|
| %R | 24-hour HH:MM time, equivalent to %H:%i | 22:33 |
|
||||||
| %s | second (00-59) | 44 |
|
| %s | second (00-59) | 44 |
|
||||||
| %S | second (00-59) | 44 |
|
| %S | second (00-59) | 44 |
|
||||||
|
@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed
|
|||||||
If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead.
|
If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead.
|
||||||
If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))).
|
If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))).
|
||||||
|
|
||||||
## sipHash64 {#hash_functions-siphash64}
|
## sipHash64 (#hash_functions-siphash64)
|
||||||
|
|
||||||
Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value.
|
Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value.
|
||||||
|
|
||||||
|
@ -1637,17 +1637,26 @@ try
|
|||||||
global_context->initializeTraceCollector();
|
global_context->initializeTraceCollector();
|
||||||
|
|
||||||
/// Set up server-wide memory profiler (for total memory tracker).
|
/// Set up server-wide memory profiler (for total memory tracker).
|
||||||
UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0);
|
if (server_settings.total_memory_profiler_step)
|
||||||
if (total_memory_profiler_step)
|
|
||||||
{
|
{
|
||||||
total_memory_tracker.setProfilerStep(total_memory_profiler_step);
|
total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step);
|
||||||
}
|
}
|
||||||
|
|
||||||
double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0);
|
if (server_settings.total_memory_tracker_sample_probability > 0.0)
|
||||||
if (total_memory_tracker_sample_probability > 0.0)
|
|
||||||
{
|
{
|
||||||
total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability);
|
total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (server_settings.total_memory_profiler_sample_min_allocation_size)
|
||||||
|
{
|
||||||
|
total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (server_settings.total_memory_profiler_sample_max_allocation_size)
|
||||||
|
{
|
||||||
|
total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
@ -3,7 +3,6 @@
|
|||||||
#include "CurrentThread.h"
|
#include "CurrentThread.h"
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include <Common/ThreadStatus.h>
|
#include <Common/ThreadStatus.h>
|
||||||
#include <Common/TaskStatsInfoGetter.h>
|
|
||||||
#include <Interpreters/ProcessList.h>
|
#include <Interpreters/ProcessList.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <base/getThreadId.h>
|
#include <base/getThreadId.h>
|
||||||
|
@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
|
|||||||
}
|
}
|
||||||
|
|
||||||
std::bernoulli_distribution sample(sample_probability);
|
std::bernoulli_distribution sample(sample_probability);
|
||||||
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
|
if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng)))
|
||||||
{
|
{
|
||||||
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
||||||
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size});
|
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size});
|
||||||
@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size)
|
|||||||
}
|
}
|
||||||
|
|
||||||
std::bernoulli_distribution sample(sample_probability);
|
std::bernoulli_distribution sample(sample_probability);
|
||||||
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
|
if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng)))
|
||||||
{
|
{
|
||||||
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
||||||
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size});
|
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size});
|
||||||
@ -534,6 +534,12 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value)
|
|||||||
;
|
;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool MemoryTracker::isSizeOkForSampling(UInt64 size) const
|
||||||
|
{
|
||||||
|
/// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation
|
||||||
|
return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes);
|
||||||
|
}
|
||||||
|
|
||||||
bool canEnqueueBackgroundTask()
|
bool canEnqueueBackgroundTask()
|
||||||
{
|
{
|
||||||
auto limit = background_memory_tracker.getSoftLimit();
|
auto limit = background_memory_tracker.getSoftLimit();
|
||||||
|
@ -67,6 +67,12 @@ private:
|
|||||||
/// To randomly sample allocations and deallocations in trace_log.
|
/// To randomly sample allocations and deallocations in trace_log.
|
||||||
double sample_probability = 0;
|
double sample_probability = 0;
|
||||||
|
|
||||||
|
/// Randomly sample allocations only larger or equal to this size
|
||||||
|
UInt64 min_allocation_size_bytes = 0;
|
||||||
|
|
||||||
|
/// Randomly sample allocations only smaller or equal to this size
|
||||||
|
UInt64 max_allocation_size_bytes = 0;
|
||||||
|
|
||||||
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
|
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
|
||||||
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
|
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
|
||||||
std::atomic<MemoryTracker *> parent {};
|
std::atomic<MemoryTracker *> parent {};
|
||||||
@ -88,6 +94,8 @@ private:
|
|||||||
|
|
||||||
void setOrRaiseProfilerLimit(Int64 value);
|
void setOrRaiseProfilerLimit(Int64 value);
|
||||||
|
|
||||||
|
bool isSizeOkForSampling(UInt64 size) const;
|
||||||
|
|
||||||
/// allocImpl(...) and free(...) should not be used directly
|
/// allocImpl(...) and free(...) should not be used directly
|
||||||
friend struct CurrentMemoryTracker;
|
friend struct CurrentMemoryTracker;
|
||||||
void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr);
|
void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr);
|
||||||
@ -165,6 +173,16 @@ public:
|
|||||||
sample_probability = value;
|
sample_probability = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void setSampleMinAllocationSize(UInt64 value)
|
||||||
|
{
|
||||||
|
min_allocation_size_bytes = value;
|
||||||
|
}
|
||||||
|
|
||||||
|
void setSampleMaxAllocationSize(UInt64 value)
|
||||||
|
{
|
||||||
|
max_allocation_size_bytes = value;
|
||||||
|
}
|
||||||
|
|
||||||
void setProfilerStep(Int64 value)
|
void setProfilerStep(Int64 value)
|
||||||
{
|
{
|
||||||
profiler_step = value;
|
profiler_step = value;
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include "TaskStatsInfoGetter.h"
|
#include "NetlinkMetricsProvider.h"
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <base/defines.h>
|
#include <base/defines.h>
|
||||||
#include <base/types.h>
|
#include <base/types.h>
|
||||||
@ -200,7 +200,7 @@ bool checkPermissionsImpl()
|
|||||||
if (!res)
|
if (!res)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// Check that we can successfully initialize TaskStatsInfoGetter.
|
/// Check that we can successfully initialize NetlinkMetricsProvider.
|
||||||
/// It will ask about family id through Netlink.
|
/// It will ask about family id through Netlink.
|
||||||
/// On some LXC containers we have capability but we still cannot use Netlink.
|
/// On some LXC containers we have capability but we still cannot use Netlink.
|
||||||
/// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result.
|
/// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result.
|
||||||
@ -208,7 +208,7 @@ bool checkPermissionsImpl()
|
|||||||
try
|
try
|
||||||
{
|
{
|
||||||
::taskstats stats{};
|
::taskstats stats{};
|
||||||
TaskStatsInfoGetter().getStat(stats, static_cast<pid_t>(getThreadId()));
|
NetlinkMetricsProvider().getStat(stats, static_cast<pid_t>(getThreadId()));
|
||||||
}
|
}
|
||||||
catch (const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
@ -244,14 +244,14 @@ UInt16 getFamilyId(int fd)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool TaskStatsInfoGetter::checkPermissions()
|
bool NetlinkMetricsProvider::checkPermissions()
|
||||||
{
|
{
|
||||||
static bool res = checkPermissionsImpl();
|
static bool res = checkPermissionsImpl();
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
TaskStatsInfoGetter::TaskStatsInfoGetter()
|
NetlinkMetricsProvider::NetlinkMetricsProvider()
|
||||||
{
|
{
|
||||||
netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC);
|
netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC);
|
||||||
if (netlink_socket_fd < 0)
|
if (netlink_socket_fd < 0)
|
||||||
@ -293,7 +293,7 @@ TaskStatsInfoGetter::TaskStatsInfoGetter()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const
|
void NetlinkMetricsProvider::getStat(::taskstats & out_stats, pid_t tid) const
|
||||||
{
|
{
|
||||||
NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid));
|
NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid));
|
||||||
|
|
||||||
@ -318,7 +318,7 @@ void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
TaskStatsInfoGetter::~TaskStatsInfoGetter()
|
NetlinkMetricsProvider::~NetlinkMetricsProvider()
|
||||||
{
|
{
|
||||||
if (netlink_socket_fd >= 0)
|
if (netlink_socket_fd >= 0)
|
||||||
{
|
{
|
||||||
@ -335,15 +335,15 @@ TaskStatsInfoGetter::~TaskStatsInfoGetter()
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
bool TaskStatsInfoGetter::checkPermissions()
|
bool NetlinkMetricsProvider::checkPermissions()
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
TaskStatsInfoGetter::TaskStatsInfoGetter() = default;
|
NetlinkMetricsProvider::NetlinkMetricsProvider() = default;
|
||||||
TaskStatsInfoGetter::~TaskStatsInfoGetter() = default;
|
NetlinkMetricsProvider::~NetlinkMetricsProvider() = default;
|
||||||
|
|
||||||
void TaskStatsInfoGetter::getStat(::taskstats &, pid_t) const
|
void NetlinkMetricsProvider::getStat(::taskstats &, pid_t) const
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
@ -15,11 +15,11 @@ namespace DB
|
|||||||
///
|
///
|
||||||
/// [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101
|
/// [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101
|
||||||
///
|
///
|
||||||
class TaskStatsInfoGetter : private boost::noncopyable
|
class NetlinkMetricsProvider : private boost::noncopyable
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
TaskStatsInfoGetter();
|
NetlinkMetricsProvider();
|
||||||
~TaskStatsInfoGetter();
|
~NetlinkMetricsProvider();
|
||||||
|
|
||||||
void getStat(::taskstats & out_stats, pid_t tid) const;
|
void getStat(::taskstats & out_stats, pid_t tid) const;
|
||||||
|
|
@ -1,6 +1,7 @@
|
|||||||
#include <limits>
|
#include <limits>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/PODArray.h>
|
#include <Common/PODArray.h>
|
||||||
|
#include <Common/checkStackSize.h>
|
||||||
#include <Common/OptimizedRegularExpression.h>
|
#include <Common/OptimizedRegularExpression.h>
|
||||||
|
|
||||||
#define MIN_LENGTH_FOR_STRSTR 3
|
#define MIN_LENGTH_FOR_STRSTR 3
|
||||||
@ -50,6 +51,8 @@ const char * analyzeImpl(
|
|||||||
bool & is_trivial,
|
bool & is_trivial,
|
||||||
Literals & global_alternatives)
|
Literals & global_alternatives)
|
||||||
{
|
{
|
||||||
|
checkStackSize();
|
||||||
|
|
||||||
/** The expression is trivial if all the metacharacters in it are escaped.
|
/** The expression is trivial if all the metacharacters in it are escaped.
|
||||||
* The non-alternative string is
|
* The non-alternative string is
|
||||||
* a string outside parentheses,
|
* a string outside parentheses,
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#if defined(OS_LINUX)
|
#if defined(OS_LINUX)
|
||||||
|
|
||||||
#include "TaskStatsInfoGetter.h"
|
#include "NetlinkMetricsProvider.h"
|
||||||
#include "ProcfsMetricsProvider.h"
|
#include "ProcfsMetricsProvider.h"
|
||||||
#include "hasLinuxCapability.h"
|
#include "hasLinuxCapability.h"
|
||||||
|
|
||||||
@ -99,7 +99,7 @@ TasksStatsCounters::MetricsProvider TasksStatsCounters::findBestAvailableProvide
|
|||||||
static std::optional<MetricsProvider> provider =
|
static std::optional<MetricsProvider> provider =
|
||||||
[]() -> MetricsProvider
|
[]() -> MetricsProvider
|
||||||
{
|
{
|
||||||
if (TaskStatsInfoGetter::checkPermissions())
|
if (NetlinkMetricsProvider::checkPermissions())
|
||||||
{
|
{
|
||||||
return MetricsProvider::Netlink;
|
return MetricsProvider::Netlink;
|
||||||
}
|
}
|
||||||
@ -119,7 +119,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p
|
|||||||
switch (provider)
|
switch (provider)
|
||||||
{
|
{
|
||||||
case MetricsProvider::Netlink:
|
case MetricsProvider::Netlink:
|
||||||
stats_getter = [metrics_provider = std::make_shared<TaskStatsInfoGetter>(), tid]()
|
stats_getter = [metrics_provider = std::make_shared<NetlinkMetricsProvider>(), tid]()
|
||||||
{
|
{
|
||||||
::taskstats result{};
|
::taskstats result{};
|
||||||
metrics_provider->getStat(result, static_cast<pid_t>(tid));
|
metrics_provider->getStat(result, static_cast<pid_t>(tid));
|
||||||
|
@ -81,8 +81,12 @@ namespace DB
|
|||||||
M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
|
M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
|
||||||
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
|
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
|
||||||
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
|
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
|
||||||
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0)
|
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \
|
||||||
|
\
|
||||||
|
M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \
|
||||||
|
M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
|
||||||
|
M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||||
|
M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0)
|
||||||
|
|
||||||
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)
|
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)
|
||||||
|
|
||||||
|
@ -427,7 +427,9 @@ class IColumn;
|
|||||||
M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \
|
M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \
|
||||||
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
|
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
|
||||||
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
|
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
|
||||||
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
|
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
|
||||||
|
M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||||
|
M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||||
M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \
|
M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \
|
||||||
\
|
\
|
||||||
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \
|
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \
|
||||||
@ -761,7 +763,7 @@ class IColumn;
|
|||||||
/** Experimental functions */ \
|
/** Experimental functions */ \
|
||||||
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
|
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
|
||||||
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
|
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
|
||||||
M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \
|
M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \
|
||||||
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
|
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
|
||||||
M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \
|
M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \
|
||||||
M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \
|
M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \
|
||||||
|
@ -114,7 +114,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
|
|||||||
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
||||||
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
||||||
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
||||||
/// (TaskStatsInfoGetter has about 500K RPS).
|
/// (NetlinkMetricsProvider has about 500K RPS).
|
||||||
Stopwatch watch(CLOCK_MONOTONIC);
|
Stopwatch watch(CLOCK_MONOTONIC);
|
||||||
|
|
||||||
SCOPE_EXIT({
|
SCOPE_EXIT({
|
||||||
|
@ -21,7 +21,6 @@ list (APPEND PUBLIC_LIBS
|
|||||||
dbms
|
dbms
|
||||||
ch_contrib::metrohash
|
ch_contrib::metrohash
|
||||||
ch_contrib::murmurhash
|
ch_contrib::murmurhash
|
||||||
ch_contrib::hashidsxx
|
|
||||||
ch_contrib::morton_nd
|
ch_contrib::morton_nd
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -49,6 +49,9 @@ struct CountSubstringsImpl
|
|||||||
/// FIXME: suboptimal
|
/// FIXME: suboptimal
|
||||||
memset(&res[0], 0, res.size() * sizeof(res[0]));
|
memset(&res[0], 0, res.size() * sizeof(res[0]));
|
||||||
|
|
||||||
|
if (needle.empty())
|
||||||
|
return; // Return all zeros
|
||||||
|
|
||||||
/// Current index in the array of strings.
|
/// Current index in the array of strings.
|
||||||
size_t i = 0;
|
size_t i = 0;
|
||||||
|
|
||||||
@ -223,6 +226,8 @@ struct CountSubstringsImpl
|
|||||||
const char * needle_beg = reinterpret_cast<const char *>(&needle_data[prev_needle_offset]);
|
const char * needle_beg = reinterpret_cast<const char *>(&needle_data[prev_needle_offset]);
|
||||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||||
|
|
||||||
|
if (needle_size > 0)
|
||||||
|
{
|
||||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size);
|
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size);
|
||||||
|
|
||||||
const UInt8 * end = reinterpret_cast<const UInt8 *>(haystack.data() + haystack.size());
|
const UInt8 * end = reinterpret_cast<const UInt8 *>(haystack.data() + haystack.size());
|
||||||
@ -235,6 +240,7 @@ struct CountSubstringsImpl
|
|||||||
beg = pos + needle_size;
|
beg = pos + needle_size;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
prev_needle_offset = needle_offsets[i];
|
prev_needle_offset = needle_offsets[i];
|
||||||
}
|
}
|
||||||
|
@ -1,12 +0,0 @@
|
|||||||
#include "FunctionHashID.h"
|
|
||||||
#include <Functions/FunctionFactory.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
REGISTER_FUNCTION(HashID)
|
|
||||||
{
|
|
||||||
factory.registerFunction<FunctionHashID>();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,170 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include "config.h"
|
|
||||||
|
|
||||||
#include <hashids.h>
|
|
||||||
|
|
||||||
#include <Columns/ColumnString.h>
|
|
||||||
#include <Columns/ColumnsNumber.h>
|
|
||||||
#include <DataTypes/DataTypeString.h>
|
|
||||||
#include <Functions/FunctionFactory.h>
|
|
||||||
#include <Functions/FunctionHelpers.h>
|
|
||||||
#include <Functions/IFunction.h>
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
|
|
||||||
#include <functional>
|
|
||||||
#include <initializer_list>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int BAD_ARGUMENTS;
|
|
||||||
extern const int ILLEGAL_COLUMN;
|
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
||||||
extern const int SUPPORT_IS_DISABLED;
|
|
||||||
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
|
|
||||||
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
|
||||||
}
|
|
||||||
|
|
||||||
// hashid(string, salt)
|
|
||||||
class FunctionHashID : public IFunction
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
static constexpr auto name = "hashid";
|
|
||||||
|
|
||||||
static FunctionPtr create(ContextPtr context)
|
|
||||||
{
|
|
||||||
if (!context->getSettingsRef().allow_experimental_hash_functions)
|
|
||||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
|
||||||
"Hashing function '{}' is experimental. Set `allow_experimental_hash_functions` setting to enable it", name);
|
|
||||||
|
|
||||||
return std::make_shared<FunctionHashID>();
|
|
||||||
}
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
|
||||||
|
|
||||||
bool isVariadic() const override { return true; }
|
|
||||||
|
|
||||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
||||||
{
|
|
||||||
if (arguments.empty())
|
|
||||||
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least one argument", getName());
|
|
||||||
|
|
||||||
const auto & id_col = arguments[0];
|
|
||||||
if (!isUnsignedInteger(id_col.type))
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
||||||
"First argument of function {} must be unsigned integer, got {}",
|
|
||||||
getName(),
|
|
||||||
arguments[0].type->getName());
|
|
||||||
|
|
||||||
if (arguments.size() > 1)
|
|
||||||
{
|
|
||||||
const auto & hash_col = arguments[1];
|
|
||||||
if (!isString(hash_col.type))
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
||||||
"Second argument of function {} must be String, got {}",
|
|
||||||
getName(),
|
|
||||||
arguments[1].type->getName());
|
|
||||||
}
|
|
||||||
|
|
||||||
if (arguments.size() > 2)
|
|
||||||
{
|
|
||||||
const auto & min_length_col = arguments[2];
|
|
||||||
if (!isUInt8(min_length_col.type))
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
||||||
"Third argument of function {} must be UInt8, got {}",
|
|
||||||
getName(),
|
|
||||||
arguments[2].type->getName());
|
|
||||||
}
|
|
||||||
|
|
||||||
if (arguments.size() > 3)
|
|
||||||
{
|
|
||||||
const auto & alphabet_col = arguments[3];
|
|
||||||
if (!isString(alphabet_col.type))
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
||||||
"Fourth argument of function {} must be String, got {}",
|
|
||||||
getName(),
|
|
||||||
arguments[3].type->getName());
|
|
||||||
}
|
|
||||||
|
|
||||||
if (arguments.size() > 4)
|
|
||||||
{
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
|
|
||||||
"Function {} expect no more than four arguments (integer, salt, min_length, optional_alphabet), got {}",
|
|
||||||
getName(),
|
|
||||||
arguments.size());
|
|
||||||
}
|
|
||||||
|
|
||||||
return std::make_shared<DataTypeString>();
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
||||||
{
|
|
||||||
const auto & numcolumn = arguments[0].column;
|
|
||||||
|
|
||||||
if (checkAndGetColumn<ColumnUInt8>(numcolumn.get()) || checkAndGetColumn<ColumnUInt16>(numcolumn.get())
|
|
||||||
|| checkAndGetColumn<ColumnUInt32>(numcolumn.get()) || checkAndGetColumn<ColumnUInt64>(numcolumn.get()))
|
|
||||||
{
|
|
||||||
std::string salt;
|
|
||||||
UInt8 min_length = 0;
|
|
||||||
std::string alphabet;
|
|
||||||
|
|
||||||
if (arguments.size() >= 4)
|
|
||||||
{
|
|
||||||
const auto & alphabetcolumn = arguments[3].column;
|
|
||||||
if (const auto * alpha_col = checkAndGetColumnConst<ColumnString>(alphabetcolumn.get()))
|
|
||||||
{
|
|
||||||
alphabet = alpha_col->getValue<String>();
|
|
||||||
if (alphabet.find('\0') != std::string::npos)
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Custom alphabet must not contain null character");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
|
||||||
alphabet.assign(DEFAULT_ALPHABET);
|
|
||||||
|
|
||||||
if (arguments.size() >= 3)
|
|
||||||
{
|
|
||||||
const auto & minlengthcolumn = arguments[2].column;
|
|
||||||
if (const auto * min_length_col = checkAndGetColumnConst<ColumnUInt8>(minlengthcolumn.get()))
|
|
||||||
min_length = min_length_col->getValue<UInt8>();
|
|
||||||
}
|
|
||||||
|
|
||||||
if (arguments.size() >= 2)
|
|
||||||
{
|
|
||||||
const auto & saltcolumn = arguments[1].column;
|
|
||||||
if (const auto * salt_col = checkAndGetColumnConst<ColumnString>(saltcolumn.get()))
|
|
||||||
salt = salt_col->getValue<String>();
|
|
||||||
}
|
|
||||||
|
|
||||||
hashidsxx::Hashids hash(salt, min_length, alphabet);
|
|
||||||
|
|
||||||
auto col_res = ColumnString::create();
|
|
||||||
|
|
||||||
for (size_t i = 0; i < input_rows_count; ++i)
|
|
||||||
{
|
|
||||||
col_res->insert(hash.encode({numcolumn->getUInt(i)}));
|
|
||||||
}
|
|
||||||
|
|
||||||
return col_res;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function hashid",
|
|
||||||
arguments[0].column->getName());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -79,51 +79,28 @@ namespace impl
|
|||||||
UInt64 key1 = 0;
|
UInt64 key1 = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct SipHashKeyColumns
|
static SipHashKey parseSipHashKey(const ColumnWithTypeAndName & key)
|
||||||
{
|
{
|
||||||
ColumnPtr key0;
|
SipHashKey ret{};
|
||||||
ColumnPtr key1;
|
|
||||||
bool is_const;
|
|
||||||
|
|
||||||
size_t size() const
|
const auto * tuple = checkAndGetColumn<ColumnTuple>(key.column.get());
|
||||||
{
|
|
||||||
assert(key0 && key1);
|
|
||||||
assert(key0->size() == key1->size());
|
|
||||||
return key0->size();
|
|
||||||
}
|
|
||||||
SipHashKey getKey(size_t i) const
|
|
||||||
{
|
|
||||||
if (is_const)
|
|
||||||
i = 0;
|
|
||||||
const auto & key0data = assert_cast<const ColumnUInt64 &>(*key0).getData();
|
|
||||||
const auto & key1data = assert_cast<const ColumnUInt64 &>(*key1).getData();
|
|
||||||
return {key0data[i], key1data[i]};
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
static SipHashKeyColumns parseSipHashKeyColumns(const ColumnWithTypeAndName & key)
|
|
||||||
{
|
|
||||||
const ColumnTuple * tuple = nullptr;
|
|
||||||
const auto * column = key.column.get();
|
|
||||||
bool is_const = false;
|
|
||||||
if (isColumnConst(*column))
|
|
||||||
{
|
|
||||||
is_const = true;
|
|
||||||
tuple = checkAndGetColumnConstData<ColumnTuple>(column);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
tuple = checkAndGetColumn<ColumnTuple>(column);
|
|
||||||
if (!tuple)
|
if (!tuple)
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple");
|
||||||
|
|
||||||
if (tuple->tupleSize() != 2)
|
if (tuple->tupleSize() != 2)
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64");
|
||||||
|
|
||||||
SipHashKeyColumns ret{tuple->getColumnPtr(0), tuple->getColumnPtr(1), is_const};
|
if (tuple->empty())
|
||||||
assert(ret.key0);
|
return ret;
|
||||||
if (!checkColumn<ColumnUInt64>(*ret.key0))
|
|
||||||
|
if (const auto * key0col = checkAndGetColumn<ColumnUInt64>(&(tuple->getColumn(0))))
|
||||||
|
ret.key0 = key0col->get64(0);
|
||||||
|
else
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64");
|
||||||
assert(ret.key1);
|
|
||||||
if (!checkColumn<ColumnUInt64>(*ret.key1))
|
if (const auto * key1col = checkAndGetColumn<ColumnUInt64>(&(tuple->getColumn(1))))
|
||||||
|
ret.key1 = key1col->get64(0);
|
||||||
|
else
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64");
|
||||||
|
|
||||||
return ret;
|
return ret;
|
||||||
@ -352,10 +329,8 @@ struct SipHash64KeyedImpl
|
|||||||
static constexpr auto name = "sipHash64Keyed";
|
static constexpr auto name = "sipHash64Keyed";
|
||||||
using ReturnType = UInt64;
|
using ReturnType = UInt64;
|
||||||
using Key = impl::SipHashKey;
|
using Key = impl::SipHashKey;
|
||||||
using KeyColumns = impl::SipHashKeyColumns;
|
|
||||||
|
|
||||||
static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); }
|
static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); }
|
||||||
static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); }
|
|
||||||
|
|
||||||
static UInt64 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash64Keyed(key.key0, key.key1, begin, size); }
|
static UInt64 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash64Keyed(key.key0, key.key1, begin, size); }
|
||||||
|
|
||||||
@ -396,10 +371,8 @@ struct SipHash128KeyedImpl
|
|||||||
static constexpr auto name = "sipHash128Keyed";
|
static constexpr auto name = "sipHash128Keyed";
|
||||||
using ReturnType = UInt128;
|
using ReturnType = UInt128;
|
||||||
using Key = impl::SipHashKey;
|
using Key = impl::SipHashKey;
|
||||||
using KeyColumns = impl::SipHashKeyColumns;
|
|
||||||
|
|
||||||
static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); }
|
static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); }
|
||||||
static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); }
|
|
||||||
|
|
||||||
static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash128Keyed(key.key0, key.key1, begin, size); }
|
static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash128Keyed(key.key0, key.key1, begin, size); }
|
||||||
|
|
||||||
@ -425,43 +398,13 @@ struct SipHash128ReferenceImpl
|
|||||||
|
|
||||||
using ReturnType = UInt128;
|
using ReturnType = UInt128;
|
||||||
|
|
||||||
static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc<UInt128, SipHash128ReferenceImpl>(h1, h2); }
|
static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc<UInt128, SipHash128Impl>(h1, h2); }
|
||||||
|
|
||||||
static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); }
|
static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); }
|
||||||
|
|
||||||
static constexpr bool use_int_hash_for_pods = false;
|
static constexpr bool use_int_hash_for_pods = false;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct SipHash128ReferenceKeyedImpl
|
|
||||||
{
|
|
||||||
static constexpr auto name = "sipHash128ReferenceKeyed";
|
|
||||||
using ReturnType = UInt128;
|
|
||||||
using Key = impl::SipHashKey;
|
|
||||||
using KeyColumns = impl::SipHashKeyColumns;
|
|
||||||
|
|
||||||
static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); }
|
|
||||||
static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); }
|
|
||||||
|
|
||||||
static UInt128 applyKeyed(const Key & key, const char * begin, size_t size)
|
|
||||||
{
|
|
||||||
return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size);
|
|
||||||
}
|
|
||||||
|
|
||||||
static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2)
|
|
||||||
{
|
|
||||||
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
|
|
||||||
UInt128 tmp;
|
|
||||||
reverseMemcpy(&tmp, &h1, sizeof(UInt128));
|
|
||||||
h1 = tmp;
|
|
||||||
reverseMemcpy(&tmp, &h2, sizeof(UInt128));
|
|
||||||
h2 = tmp;
|
|
||||||
#endif
|
|
||||||
UInt128 hashes[] = {h1, h2};
|
|
||||||
return applyKeyed(key, reinterpret_cast<const char *>(hashes), 2 * sizeof(UInt128));
|
|
||||||
}
|
|
||||||
|
|
||||||
static constexpr bool use_int_hash_for_pods = false;
|
|
||||||
};
|
|
||||||
|
|
||||||
/** Why we need MurmurHash2?
|
/** Why we need MurmurHash2?
|
||||||
* MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash.
|
* MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash.
|
||||||
@ -1080,7 +1023,7 @@ private:
|
|||||||
|
|
||||||
DECLARE_MULTITARGET_CODE(
|
DECLARE_MULTITARGET_CODE(
|
||||||
|
|
||||||
template <typename Impl, bool Keyed, typename KeyType, typename KeyColumnsType>
|
template <typename Impl, bool Keyed, typename KeyType>
|
||||||
class FunctionAnyHash : public IFunction
|
class FunctionAnyHash : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -1090,12 +1033,9 @@ private:
|
|||||||
using ToType = typename Impl::ReturnType;
|
using ToType = typename Impl::ReturnType;
|
||||||
|
|
||||||
template <typename FromType, bool first>
|
template <typename FromType, bool first>
|
||||||
void executeIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
void executeIntType(const KeyType & key, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
||||||
{
|
{
|
||||||
using ColVecType = ColumnVectorOrDecimal<FromType>;
|
using ColVecType = ColumnVectorOrDecimal<FromType>;
|
||||||
KeyType key{};
|
|
||||||
if constexpr (Keyed)
|
|
||||||
key = Impl::getKey(key_cols, 0);
|
|
||||||
|
|
||||||
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
|
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
|
||||||
{
|
{
|
||||||
@ -1104,9 +1044,6 @@ private:
|
|||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
{
|
||||||
ToType hash;
|
ToType hash;
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
|
|
||||||
if constexpr (Impl::use_int_hash_for_pods)
|
if constexpr (Impl::use_int_hash_for_pods)
|
||||||
{
|
{
|
||||||
@ -1140,14 +1077,6 @@ private:
|
|||||||
}
|
}
|
||||||
else if (auto col_from_const = checkAndGetColumnConst<ColVecType>(column))
|
else if (auto col_from_const = checkAndGetColumnConst<ColVecType>(column))
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
{
|
|
||||||
if (!key_cols.is_const)
|
|
||||||
{
|
|
||||||
ColumnPtr full_column = col_from_const->convertToFullColumn();
|
|
||||||
return executeIntType<FromType, first>(key_cols, full_column.get(), vec_to);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
auto value = col_from_const->template getValue<FromType>();
|
auto value = col_from_const->template getValue<FromType>();
|
||||||
ToType hash;
|
ToType hash;
|
||||||
|
|
||||||
@ -1178,28 +1107,18 @@ private:
|
|||||||
if constexpr (first)
|
if constexpr (first)
|
||||||
vec_to.assign(size, hash);
|
vec_to.assign(size, hash);
|
||||||
else
|
else
|
||||||
{
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
vec_to[i] = combineHashes(key, vec_to[i], hash);
|
vec_to[i] = combineHashes(key, vec_to[i], hash);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
||||||
column->getName(), getName());
|
column->getName(), getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename FromType, bool first>
|
template <typename FromType, bool first>
|
||||||
void executeBigIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
void executeBigIntType(const KeyType & key, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
||||||
{
|
{
|
||||||
using ColVecType = ColumnVectorOrDecimal<FromType>;
|
using ColVecType = ColumnVectorOrDecimal<FromType>;
|
||||||
KeyType key{};
|
|
||||||
if constexpr (Keyed)
|
|
||||||
key = Impl::getKey(key_cols, 0);
|
|
||||||
|
|
||||||
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
|
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
|
||||||
{
|
{
|
||||||
@ -1208,9 +1127,6 @@ private:
|
|||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
{
|
||||||
ToType hash;
|
ToType hash;
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
if constexpr (std::endian::native == std::endian::little)
|
if constexpr (std::endian::native == std::endian::little)
|
||||||
hash = apply(key, reinterpret_cast<const char *>(&vec_from[i]), sizeof(vec_from[i]));
|
hash = apply(key, reinterpret_cast<const char *>(&vec_from[i]), sizeof(vec_from[i]));
|
||||||
else
|
else
|
||||||
@ -1227,14 +1143,6 @@ private:
|
|||||||
}
|
}
|
||||||
else if (auto col_from_const = checkAndGetColumnConst<ColVecType>(column))
|
else if (auto col_from_const = checkAndGetColumnConst<ColVecType>(column))
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
{
|
|
||||||
if (!key_cols.is_const)
|
|
||||||
{
|
|
||||||
ColumnPtr full_column = col_from_const->convertToFullColumn();
|
|
||||||
return executeBigIntType<FromType, first>(key_cols, full_column.get(), vec_to);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
auto value = col_from_const->template getValue<FromType>();
|
auto value = col_from_const->template getValue<FromType>();
|
||||||
|
|
||||||
ToType hash;
|
ToType hash;
|
||||||
@ -1250,32 +1158,19 @@ private:
|
|||||||
if constexpr (first)
|
if constexpr (first)
|
||||||
vec_to.assign(size, hash);
|
vec_to.assign(size, hash);
|
||||||
else
|
else
|
||||||
{
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
vec_to[i] = combineHashes(key, vec_to[i], hash);
|
vec_to[i] = combineHashes(key, vec_to[i], hash);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
||||||
column->getName(), getName());
|
column->getName(), getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
template <bool first>
|
template <bool first>
|
||||||
void executeGeneric(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
void executeGeneric(const KeyType & key, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
||||||
{
|
{
|
||||||
KeyType key{};
|
|
||||||
if constexpr (Keyed)
|
|
||||||
key = Impl::getKey(key_cols, 0);
|
|
||||||
for (size_t i = 0, size = column->size(); i < size; ++i)
|
for (size_t i = 0, size = column->size(); i < size; ++i)
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
StringRef bytes = column->getDataAt(i);
|
StringRef bytes = column->getDataAt(i);
|
||||||
const ToType hash = apply(key, bytes.data, bytes.size);
|
const ToType hash = apply(key, bytes.data, bytes.size);
|
||||||
if constexpr (first)
|
if constexpr (first)
|
||||||
@ -1286,11 +1181,8 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <bool first>
|
template <bool first>
|
||||||
void executeString(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
void executeString(const KeyType & key, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
||||||
{
|
{
|
||||||
KeyType key{};
|
|
||||||
if constexpr (Keyed)
|
|
||||||
key = Impl::getKey(key_cols, 0);
|
|
||||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(column))
|
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(column))
|
||||||
{
|
{
|
||||||
const typename ColumnString::Chars & data = col_from->getChars();
|
const typename ColumnString::Chars & data = col_from->getChars();
|
||||||
@ -1300,9 +1192,6 @@ private:
|
|||||||
ColumnString::Offset current_offset = 0;
|
ColumnString::Offset current_offset = 0;
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
const ToType hash = apply(key,
|
const ToType hash = apply(key,
|
||||||
reinterpret_cast<const char *>(&data[current_offset]),
|
reinterpret_cast<const char *>(&data[current_offset]),
|
||||||
offsets[i] - current_offset - 1);
|
offsets[i] - current_offset - 1);
|
||||||
@ -1323,9 +1212,6 @@ private:
|
|||||||
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
const ToType hash = apply(key, reinterpret_cast<const char *>(&data[i * n]), n);
|
const ToType hash = apply(key, reinterpret_cast<const char *>(&data[i * n]), n);
|
||||||
if constexpr (first)
|
if constexpr (first)
|
||||||
vec_to[i] = hash;
|
vec_to[i] = hash;
|
||||||
@ -1335,14 +1221,6 @@ private:
|
|||||||
}
|
}
|
||||||
else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column))
|
else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column))
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
{
|
|
||||||
if (!key_cols.is_const)
|
|
||||||
{
|
|
||||||
ColumnPtr full_column = col_from_const->convertToFullColumn();
|
|
||||||
return executeString<first>(key_cols, full_column.get(), vec_to);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
String value = col_from_const->getValue<String>();
|
String value = col_from_const->getValue<String>();
|
||||||
const ToType hash = apply(key, value.data(), value.size());
|
const ToType hash = apply(key, value.data(), value.size());
|
||||||
const size_t size = vec_to.size();
|
const size_t size = vec_to.size();
|
||||||
@ -1350,23 +1228,16 @@ private:
|
|||||||
if constexpr (first)
|
if constexpr (first)
|
||||||
vec_to.assign(size, hash);
|
vec_to.assign(size, hash);
|
||||||
else
|
else
|
||||||
{
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
vec_to[i] = combineHashes(key, vec_to[i], hash);
|
vec_to[i] = combineHashes(key, vec_to[i], hash);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
||||||
column->getName(), getName());
|
column->getName(), getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
template <bool first>
|
template <bool first>
|
||||||
void executeArray(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
void executeArray(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
|
||||||
{
|
{
|
||||||
const IDataType * nested_type = typeid_cast<const DataTypeArray &>(*type).getNestedType().get();
|
const IDataType * nested_type = typeid_cast<const DataTypeArray &>(*type).getNestedType().get();
|
||||||
|
|
||||||
@ -1378,19 +1249,13 @@ private:
|
|||||||
|
|
||||||
typename ColumnVector<ToType>::Container vec_temp(nested_size);
|
typename ColumnVector<ToType>::Container vec_temp(nested_size);
|
||||||
bool nested_is_first = true;
|
bool nested_is_first = true;
|
||||||
executeForArgument(key_cols, nested_type, nested_column, vec_temp, nested_is_first);
|
executeForArgument(key, nested_type, nested_column, vec_temp, nested_is_first);
|
||||||
|
|
||||||
const size_t size = offsets.size();
|
const size_t size = offsets.size();
|
||||||
|
|
||||||
ColumnArray::Offset current_offset = 0;
|
ColumnArray::Offset current_offset = 0;
|
||||||
KeyType key{};
|
|
||||||
if constexpr (Keyed)
|
|
||||||
key = Impl::getKey(key_cols, 0);
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
{
|
{
|
||||||
if constexpr (Keyed)
|
|
||||||
if (!key_cols.is_const && i != 0)
|
|
||||||
key = Impl::getKey(key_cols, i);
|
|
||||||
ColumnArray::Offset next_offset = offsets[i];
|
ColumnArray::Offset next_offset = offsets[i];
|
||||||
|
|
||||||
ToType hash;
|
ToType hash;
|
||||||
@ -1414,7 +1279,7 @@ private:
|
|||||||
{
|
{
|
||||||
/// NOTE: here, of course, you can do without the materialization of the column.
|
/// NOTE: here, of course, you can do without the materialization of the column.
|
||||||
ColumnPtr full_column = col_from_const->convertToFullColumn();
|
ColumnPtr full_column = col_from_const->convertToFullColumn();
|
||||||
executeArray<first>(key_cols, type, full_column.get(), vec_to);
|
executeArray<first>(key, type, full_column.get(), vec_to);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
||||||
@ -1422,7 +1287,7 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <bool first>
|
template <bool first>
|
||||||
void executeAny(const KeyColumnsType & key_cols, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector<ToType>::Container & vec_to) const
|
void executeAny(const KeyType & key, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector<ToType>::Container & vec_to) const
|
||||||
{
|
{
|
||||||
WhichDataType which(from_type);
|
WhichDataType which(from_type);
|
||||||
|
|
||||||
@ -1430,45 +1295,40 @@ private:
|
|||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}",
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}",
|
||||||
icolumn->getName(), icolumn->size(), vec_to.size(), getName());
|
icolumn->getName(), icolumn->size(), vec_to.size(), getName());
|
||||||
|
|
||||||
if constexpr (Keyed)
|
if (which.isUInt8()) executeIntType<UInt8, first>(key, icolumn, vec_to);
|
||||||
if ((!key_cols.is_const && key_cols.size() != vec_to.size())
|
else if (which.isUInt16()) executeIntType<UInt16, first>(key, icolumn, vec_to);
|
||||||
|| (key_cols.is_const && key_cols.size() != 1))
|
else if (which.isUInt32()) executeIntType<UInt32, first>(key, icolumn, vec_to);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Key column size {} doesn't match result column size {} of function {}", key_cols.size(), vec_to.size(), getName());
|
else if (which.isUInt64()) executeIntType<UInt64, first>(key, icolumn, vec_to);
|
||||||
|
else if (which.isUInt128()) executeBigIntType<UInt128, first>(key, icolumn, vec_to);
|
||||||
if (which.isUInt8()) executeIntType<UInt8, first>(key_cols, icolumn, vec_to);
|
else if (which.isUInt256()) executeBigIntType<UInt256, first>(key, icolumn, vec_to);
|
||||||
else if (which.isUInt16()) executeIntType<UInt16, first>(key_cols, icolumn, vec_to);
|
else if (which.isInt8()) executeIntType<Int8, first>(key, icolumn, vec_to);
|
||||||
else if (which.isUInt32()) executeIntType<UInt32, first>(key_cols, icolumn, vec_to);
|
else if (which.isInt16()) executeIntType<Int16, first>(key, icolumn, vec_to);
|
||||||
else if (which.isUInt64()) executeIntType<UInt64, first>(key_cols, icolumn, vec_to);
|
else if (which.isInt32()) executeIntType<Int32, first>(key, icolumn, vec_to);
|
||||||
else if (which.isUInt128()) executeBigIntType<UInt128, first>(key_cols, icolumn, vec_to);
|
else if (which.isInt64()) executeIntType<Int64, first>(key, icolumn, vec_to);
|
||||||
else if (which.isUInt256()) executeBigIntType<UInt256, first>(key_cols, icolumn, vec_to);
|
else if (which.isInt128()) executeBigIntType<Int128, first>(key, icolumn, vec_to);
|
||||||
else if (which.isInt8()) executeIntType<Int8, first>(key_cols, icolumn, vec_to);
|
else if (which.isInt256()) executeBigIntType<Int256, first>(key, icolumn, vec_to);
|
||||||
else if (which.isInt16()) executeIntType<Int16, first>(key_cols, icolumn, vec_to);
|
else if (which.isUUID()) executeBigIntType<UUID, first>(key, icolumn, vec_to);
|
||||||
else if (which.isInt32()) executeIntType<Int32, first>(key_cols, icolumn, vec_to);
|
else if (which.isIPv4()) executeIntType<IPv4, first>(key, icolumn, vec_to);
|
||||||
else if (which.isInt64()) executeIntType<Int64, first>(key_cols, icolumn, vec_to);
|
else if (which.isIPv6()) executeBigIntType<IPv6, first>(key, icolumn, vec_to);
|
||||||
else if (which.isInt128()) executeBigIntType<Int128, first>(key_cols, icolumn, vec_to);
|
else if (which.isEnum8()) executeIntType<Int8, first>(key, icolumn, vec_to);
|
||||||
else if (which.isInt256()) executeBigIntType<Int256, first>(key_cols, icolumn, vec_to);
|
else if (which.isEnum16()) executeIntType<Int16, first>(key, icolumn, vec_to);
|
||||||
else if (which.isUUID()) executeBigIntType<UUID, first>(key_cols, icolumn, vec_to);
|
else if (which.isDate()) executeIntType<UInt16, first>(key, icolumn, vec_to);
|
||||||
else if (which.isIPv4()) executeIntType<IPv4, first>(key_cols, icolumn, vec_to);
|
else if (which.isDate32()) executeIntType<Int32, first>(key, icolumn, vec_to);
|
||||||
else if (which.isIPv6()) executeBigIntType<IPv6, first>(key_cols, icolumn, vec_to);
|
else if (which.isDateTime()) executeIntType<UInt32, first>(key, icolumn, vec_to);
|
||||||
else if (which.isEnum8()) executeIntType<Int8, first>(key_cols, icolumn, vec_to);
|
|
||||||
else if (which.isEnum16()) executeIntType<Int16, first>(key_cols, icolumn, vec_to);
|
|
||||||
else if (which.isDate()) executeIntType<UInt16, first>(key_cols, icolumn, vec_to);
|
|
||||||
else if (which.isDate32()) executeIntType<Int32, first>(key_cols, icolumn, vec_to);
|
|
||||||
else if (which.isDateTime()) executeIntType<UInt32, first>(key_cols, icolumn, vec_to);
|
|
||||||
/// TODO: executeIntType() for Decimal32/64 leads to incompatible result
|
/// TODO: executeIntType() for Decimal32/64 leads to incompatible result
|
||||||
else if (which.isDecimal32()) executeBigIntType<Decimal32, first>(key_cols, icolumn, vec_to);
|
else if (which.isDecimal32()) executeBigIntType<Decimal32, first>(key, icolumn, vec_to);
|
||||||
else if (which.isDecimal64()) executeBigIntType<Decimal64, first>(key_cols, icolumn, vec_to);
|
else if (which.isDecimal64()) executeBigIntType<Decimal64, first>(key, icolumn, vec_to);
|
||||||
else if (which.isDecimal128()) executeBigIntType<Decimal128, first>(key_cols, icolumn, vec_to);
|
else if (which.isDecimal128()) executeBigIntType<Decimal128, first>(key, icolumn, vec_to);
|
||||||
else if (which.isDecimal256()) executeBigIntType<Decimal256, first>(key_cols, icolumn, vec_to);
|
else if (which.isDecimal256()) executeBigIntType<Decimal256, first>(key, icolumn, vec_to);
|
||||||
else if (which.isFloat32()) executeIntType<Float32, first>(key_cols, icolumn, vec_to);
|
else if (which.isFloat32()) executeIntType<Float32, first>(key, icolumn, vec_to);
|
||||||
else if (which.isFloat64()) executeIntType<Float64, first>(key_cols, icolumn, vec_to);
|
else if (which.isFloat64()) executeIntType<Float64, first>(key, icolumn, vec_to);
|
||||||
else if (which.isString()) executeString<first>(key_cols, icolumn, vec_to);
|
else if (which.isString()) executeString<first>(key, icolumn, vec_to);
|
||||||
else if (which.isFixedString()) executeString<first>(key_cols, icolumn, vec_to);
|
else if (which.isFixedString()) executeString<first>(key, icolumn, vec_to);
|
||||||
else if (which.isArray()) executeArray<first>(key_cols, from_type, icolumn, vec_to);
|
else if (which.isArray()) executeArray<first>(key, from_type, icolumn, vec_to);
|
||||||
else executeGeneric<first>(key_cols, icolumn, vec_to);
|
else executeGeneric<first>(key, icolumn, vec_to);
|
||||||
}
|
}
|
||||||
|
|
||||||
void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, bool & is_first) const
|
void executeForArgument(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, bool & is_first) const
|
||||||
{
|
{
|
||||||
/// Flattening of tuples.
|
/// Flattening of tuples.
|
||||||
if (const ColumnTuple * tuple = typeid_cast<const ColumnTuple *>(column))
|
if (const ColumnTuple * tuple = typeid_cast<const ColumnTuple *>(column))
|
||||||
@ -1477,7 +1337,7 @@ private:
|
|||||||
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
|
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
|
||||||
size_t tuple_size = tuple_columns.size();
|
size_t tuple_size = tuple_columns.size();
|
||||||
for (size_t i = 0; i < tuple_size; ++i)
|
for (size_t i = 0; i < tuple_size; ++i)
|
||||||
executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first);
|
executeForArgument(key, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first);
|
||||||
}
|
}
|
||||||
else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData<ColumnTuple>(column))
|
else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData<ColumnTuple>(column))
|
||||||
{
|
{
|
||||||
@ -1487,24 +1347,24 @@ private:
|
|||||||
for (size_t i = 0; i < tuple_size; ++i)
|
for (size_t i = 0; i < tuple_size; ++i)
|
||||||
{
|
{
|
||||||
auto tmp = ColumnConst::create(tuple_columns[i], column->size());
|
auto tmp = ColumnConst::create(tuple_columns[i], column->size());
|
||||||
executeForArgument(key_cols, tuple_types[i].get(), tmp.get(), vec_to, is_first);
|
executeForArgument(key, tuple_types[i].get(), tmp.get(), vec_to, is_first);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (const auto * map = checkAndGetColumn<ColumnMap>(column))
|
else if (const auto * map = checkAndGetColumn<ColumnMap>(column))
|
||||||
{
|
{
|
||||||
const auto & type_map = assert_cast<const DataTypeMap &>(*type);
|
const auto & type_map = assert_cast<const DataTypeMap &>(*type);
|
||||||
executeForArgument(key_cols, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first);
|
executeForArgument(key, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first);
|
||||||
}
|
}
|
||||||
else if (const auto * const_map = checkAndGetColumnConst<ColumnMap>(column))
|
else if (const auto * const_map = checkAndGetColumnConst<ColumnMap>(column))
|
||||||
{
|
{
|
||||||
executeForArgument(key_cols, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first);
|
executeForArgument(key, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
if (is_first)
|
if (is_first)
|
||||||
executeAny<true>(key_cols, type, column, vec_to);
|
executeAny<true>(key, type, column, vec_to);
|
||||||
else
|
else
|
||||||
executeAny<false>(key_cols, type, column, vec_to);
|
executeAny<false>(key, type, column, vec_to);
|
||||||
}
|
}
|
||||||
|
|
||||||
is_first = false;
|
is_first = false;
|
||||||
@ -1535,8 +1395,6 @@ public:
|
|||||||
{
|
{
|
||||||
auto col_to = ColumnVector<ToType>::create(input_rows_count);
|
auto col_to = ColumnVector<ToType>::create(input_rows_count);
|
||||||
|
|
||||||
if (input_rows_count != 0)
|
|
||||||
{
|
|
||||||
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
||||||
|
|
||||||
/// If using a "keyed" algorithm, the first argument is the key and
|
/// If using a "keyed" algorithm, the first argument is the key and
|
||||||
@ -1550,24 +1408,26 @@ public:
|
|||||||
vec_to.assign(input_rows_count, static_cast<ToType>(0xe28dbde7fe22e41c));
|
vec_to.assign(input_rows_count, static_cast<ToType>(0xe28dbde7fe22e41c));
|
||||||
}
|
}
|
||||||
|
|
||||||
KeyColumnsType key_cols{};
|
KeyType key{};
|
||||||
if constexpr (Keyed)
|
if constexpr (Keyed)
|
||||||
if (!arguments.empty())
|
if (!arguments.empty())
|
||||||
key_cols = Impl::parseKeyColumns(arguments[0]);
|
key = Impl::parseKey(arguments[0]);
|
||||||
|
|
||||||
/// The function supports arbitrary number of arguments of arbitrary types.
|
/// The function supports arbitrary number of arguments of arbitrary types.
|
||||||
bool is_first_argument = true;
|
bool is_first_argument = true;
|
||||||
for (size_t i = first_data_argument; i < arguments.size(); ++i)
|
for (size_t i = first_data_argument; i < arguments.size(); ++i)
|
||||||
{
|
{
|
||||||
const auto & col = arguments[i];
|
const auto & col = arguments[i];
|
||||||
executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument);
|
executeForArgument(key, col.type.get(), col.column.get(), vec_to, is_first_argument);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if constexpr (std::is_same_v<ToType, UInt128>) /// backward-compatible
|
if constexpr (std::is_same_v<ToType, UInt128>) /// backward-compatible
|
||||||
{
|
{
|
||||||
auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128));
|
auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128));
|
||||||
col_to_fixed_string->getChars() = std::move(*reinterpret_cast<ColumnFixedString::Chars *>(&col_to->getData()));
|
const auto & data = col_to->getData();
|
||||||
|
auto & chars = col_to_fixed_string->getChars();
|
||||||
|
chars.resize(data.size() * sizeof(UInt128));
|
||||||
|
memcpy(chars.data(), data.data(), data.size() * sizeof(UInt128));
|
||||||
return col_to_fixed_string;
|
return col_to_fixed_string;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1593,19 +1453,17 @@ public:
|
|||||||
|
|
||||||
) // DECLARE_MULTITARGET_CODE
|
) // DECLARE_MULTITARGET_CODE
|
||||||
|
|
||||||
template <typename Impl, bool Keyed = false, typename KeyType = char, typename KeyColumnsType = char>
|
template <typename Impl, bool Keyed = false, typename KeyType = char>
|
||||||
class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash<Impl, Keyed, KeyType, KeyColumnsType>
|
class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash<Impl, Keyed, KeyType>
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit FunctionAnyHash(ContextPtr context) : selector(context)
|
explicit FunctionAnyHash(ContextPtr context) : selector(context)
|
||||||
{
|
{
|
||||||
selector
|
selector.registerImplementation<TargetArch::Default, TargetSpecific::Default::FunctionAnyHash<Impl, Keyed, KeyType>>();
|
||||||
.registerImplementation<TargetArch::Default, TargetSpecific::Default::FunctionAnyHash<Impl, Keyed, KeyType, KeyColumnsType>>();
|
|
||||||
|
|
||||||
#if USE_MULTITARGET_CODE
|
#if USE_MULTITARGET_CODE
|
||||||
selector.registerImplementation<TargetArch::AVX2, TargetSpecific::AVX2::FunctionAnyHash<Impl, Keyed, KeyType, KeyColumnsType>>();
|
selector.registerImplementation<TargetArch::AVX2, TargetSpecific::AVX2::FunctionAnyHash<Impl, Keyed, KeyType>>();
|
||||||
selector
|
selector.registerImplementation<TargetArch::AVX512F, TargetSpecific::AVX512F::FunctionAnyHash<Impl, Keyed, KeyType>>();
|
||||||
.registerImplementation<TargetArch::AVX512F, TargetSpecific::AVX512F::FunctionAnyHash<Impl, Keyed, KeyType, KeyColumnsType>>();
|
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1841,7 +1699,7 @@ struct NameIntHash32 { static constexpr auto name = "intHash32"; };
|
|||||||
struct NameIntHash64 { static constexpr auto name = "intHash64"; };
|
struct NameIntHash64 { static constexpr auto name = "intHash64"; };
|
||||||
|
|
||||||
using FunctionSipHash64 = FunctionAnyHash<SipHash64Impl>;
|
using FunctionSipHash64 = FunctionAnyHash<SipHash64Impl>;
|
||||||
using FunctionSipHash64Keyed = FunctionAnyHash<SipHash64KeyedImpl, true, SipHash64KeyedImpl::Key, SipHash64KeyedImpl::KeyColumns>;
|
using FunctionSipHash64Keyed = FunctionAnyHash<SipHash64KeyedImpl, true, SipHash64KeyedImpl::Key>;
|
||||||
using FunctionIntHash32 = FunctionIntHash<IntHash32Impl, NameIntHash32>;
|
using FunctionIntHash32 = FunctionIntHash<IntHash32Impl, NameIntHash32>;
|
||||||
using FunctionIntHash64 = FunctionIntHash<IntHash64Impl, NameIntHash64>;
|
using FunctionIntHash64 = FunctionIntHash<IntHash64Impl, NameIntHash64>;
|
||||||
#if USE_SSL
|
#if USE_SSL
|
||||||
@ -1855,10 +1713,8 @@ using FunctionSHA384 = FunctionStringHashFixedString<SHA384Impl>;
|
|||||||
using FunctionSHA512 = FunctionStringHashFixedString<SHA512Impl>;
|
using FunctionSHA512 = FunctionStringHashFixedString<SHA512Impl>;
|
||||||
#endif
|
#endif
|
||||||
using FunctionSipHash128 = FunctionAnyHash<SipHash128Impl>;
|
using FunctionSipHash128 = FunctionAnyHash<SipHash128Impl>;
|
||||||
using FunctionSipHash128Keyed = FunctionAnyHash<SipHash128KeyedImpl, true, SipHash128KeyedImpl::Key, SipHash128KeyedImpl::KeyColumns>;
|
using FunctionSipHash128Keyed = FunctionAnyHash<SipHash128KeyedImpl, true, SipHash128KeyedImpl::Key>;
|
||||||
using FunctionSipHash128Reference = FunctionAnyHash<SipHash128ReferenceImpl>;
|
using FunctionSipHash128Reference = FunctionAnyHash<SipHash128ReferenceImpl>;
|
||||||
using FunctionSipHash128ReferenceKeyed
|
|
||||||
= FunctionAnyHash<SipHash128ReferenceKeyedImpl, true, SipHash128ReferenceKeyedImpl::Key, SipHash128ReferenceKeyedImpl::KeyColumns>;
|
|
||||||
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
|
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
|
||||||
using FunctionFarmFingerprint64 = FunctionAnyHash<ImplFarmFingerprint64>;
|
using FunctionFarmFingerprint64 = FunctionAnyHash<ImplFarmFingerprint64>;
|
||||||
using FunctionFarmHash64 = FunctionAnyHash<ImplFarmHash64>;
|
using FunctionFarmHash64 = FunctionAnyHash<ImplFarmHash64>;
|
||||||
|
@ -20,11 +20,6 @@ REGISTER_FUNCTION(Hashing)
|
|||||||
.examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}},
|
.examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}},
|
||||||
.categories{"Hash"}
|
.categories{"Hash"}
|
||||||
});
|
});
|
||||||
factory.registerFunction<FunctionSipHash128ReferenceKeyed>(FunctionDocumentation{
|
|
||||||
.description = "Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument "
|
|
||||||
"instead of using a fixed key.",
|
|
||||||
.examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}},
|
|
||||||
.categories{"Hash"}});
|
|
||||||
factory.registerFunction<FunctionCityHash64>();
|
factory.registerFunction<FunctionCityHash64>();
|
||||||
factory.registerFunction<FunctionFarmFingerprint64>();
|
factory.registerFunction<FunctionFarmFingerprint64>();
|
||||||
factory.registerFunction<FunctionFarmHash64>();
|
factory.registerFunction<FunctionFarmHash64>();
|
||||||
|
@ -95,7 +95,7 @@ size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_
|
|||||||
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
||||||
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
||||||
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
||||||
/// (TaskStatsInfoGetter has about 500K RPS).
|
/// (NetlinkMetricsProvider has about 500K RPS).
|
||||||
watch.stop();
|
watch.stop();
|
||||||
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
|
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
|
||||||
|
|
||||||
|
@ -78,7 +78,7 @@ std::future<IAsynchronousReader::Result> SynchronousReader::submit(Request reque
|
|||||||
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
||||||
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
||||||
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
||||||
/// (TaskStatsInfoGetter has about 500K RPS).
|
/// (NetlinkMetricsProvider has about 500K RPS).
|
||||||
watch.stop();
|
watch.stop();
|
||||||
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
|
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
|
||||||
|
|
||||||
|
@ -73,3 +73,9 @@ target_link_libraries (snappy_read_buffer PRIVATE clickhouse_common_io)
|
|||||||
clickhouse_add_executable (hadoop_snappy_read_buffer hadoop_snappy_read_buffer.cpp)
|
clickhouse_add_executable (hadoop_snappy_read_buffer hadoop_snappy_read_buffer.cpp)
|
||||||
target_link_libraries (hadoop_snappy_read_buffer PRIVATE clickhouse_common_io)
|
target_link_libraries (hadoop_snappy_read_buffer PRIVATE clickhouse_common_io)
|
||||||
|
|
||||||
|
if (TARGET ch_contrib::hdfs)
|
||||||
|
clickhouse_add_executable (read_buffer_from_hdfs read_buffer_from_hdfs.cpp)
|
||||||
|
target_link_libraries (read_buffer_from_hdfs PRIVATE dbms ch_contrib::hdfs)
|
||||||
|
endif ()
|
||||||
|
|
||||||
|
|
||||||
|
25
src/IO/examples/read_buffer_from_hdfs.cpp
Normal file
25
src/IO/examples/read_buffer_from_hdfs.cpp
Normal file
@ -0,0 +1,25 @@
|
|||||||
|
#include <iostream>
|
||||||
|
#include <memory>
|
||||||
|
#include <string>
|
||||||
|
#include <IO/WriteBufferFromFile.h>
|
||||||
|
#include <IO/copyData.h>
|
||||||
|
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||||
|
#include <base/types.h>
|
||||||
|
#include <Common/Config/ConfigProcessor.h>
|
||||||
|
|
||||||
|
using namespace DB;
|
||||||
|
|
||||||
|
int main()
|
||||||
|
{
|
||||||
|
setenv("LIBHDFS3_CONF", "/path/to/hdfs-site.xml", true); /// NOLINT
|
||||||
|
String hdfs_uri = "hdfs://cluster_name";
|
||||||
|
String hdfs_file_path = "/path/to/hdfs/file";
|
||||||
|
ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration());
|
||||||
|
ReadSettings read_settings;
|
||||||
|
ReadBufferFromHDFS read_buffer(hdfs_uri, hdfs_file_path, *config, read_settings, 2097152UL, false);
|
||||||
|
|
||||||
|
String download_path = "./download";
|
||||||
|
WriteBufferFromFile write_buffer(download_path);
|
||||||
|
copyData(read_buffer, write_buffer);
|
||||||
|
return 0;
|
||||||
|
}
|
@ -870,13 +870,12 @@ void FileCache::loadMetadata()
|
|||||||
}
|
}
|
||||||
|
|
||||||
size_t total_size = 0;
|
size_t total_size = 0;
|
||||||
for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()};
|
for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; key_prefix_it != fs::directory_iterator();
|
||||||
key_prefix_it != fs::directory_iterator();)
|
key_prefix_it++)
|
||||||
{
|
{
|
||||||
const fs::path key_prefix_directory = key_prefix_it->path();
|
const fs::path key_prefix_directory = key_prefix_it->path();
|
||||||
key_prefix_it++;
|
|
||||||
|
|
||||||
if (!fs::is_directory(key_prefix_directory))
|
if (!key_prefix_it->is_directory())
|
||||||
{
|
{
|
||||||
if (key_prefix_directory.filename() != "status")
|
if (key_prefix_directory.filename() != "status")
|
||||||
{
|
{
|
||||||
@ -887,19 +886,19 @@ void FileCache::loadMetadata()
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (fs::is_empty(key_prefix_directory))
|
fs::directory_iterator key_it{key_prefix_directory};
|
||||||
|
if (key_it == fs::directory_iterator{})
|
||||||
{
|
{
|
||||||
LOG_DEBUG(log, "Removing empty key prefix directory: {}", key_prefix_directory.string());
|
LOG_DEBUG(log, "Removing empty key prefix directory: {}", key_prefix_directory.string());
|
||||||
fs::remove(key_prefix_directory);
|
fs::remove(key_prefix_directory);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
for (fs::directory_iterator key_it{key_prefix_directory}; key_it != fs::directory_iterator();)
|
for (/* key_it already initialized to verify emptiness */; key_it != fs::directory_iterator(); key_it++)
|
||||||
{
|
{
|
||||||
const fs::path key_directory = key_it->path();
|
const fs::path key_directory = key_it->path();
|
||||||
++key_it;
|
|
||||||
|
|
||||||
if (!fs::is_directory(key_directory))
|
if (!key_it->is_directory())
|
||||||
{
|
{
|
||||||
LOG_DEBUG(
|
LOG_DEBUG(
|
||||||
log,
|
log,
|
||||||
@ -908,7 +907,7 @@ void FileCache::loadMetadata()
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (fs::is_empty(key_directory))
|
if (fs::directory_iterator{key_directory} == fs::directory_iterator{})
|
||||||
{
|
{
|
||||||
LOG_DEBUG(log, "Removing empty key directory: {}", key_directory.string());
|
LOG_DEBUG(log, "Removing empty key directory: {}", key_directory.string());
|
||||||
fs::remove(key_directory);
|
fs::remove(key_directory);
|
||||||
|
@ -1461,15 +1461,24 @@ void Context::addQueryAccessInfo(
|
|||||||
void Context::addQueryAccessInfo(const Names & partition_names)
|
void Context::addQueryAccessInfo(const Names & partition_names)
|
||||||
{
|
{
|
||||||
if (isGlobalContext())
|
if (isGlobalContext())
|
||||||
{
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info");
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info");
|
||||||
}
|
|
||||||
|
|
||||||
std::lock_guard<std::mutex> lock(query_access_info.mutex);
|
std::lock_guard<std::mutex> lock(query_access_info.mutex);
|
||||||
for (const auto & partition_name : partition_names)
|
for (const auto & partition_name : partition_names)
|
||||||
{
|
|
||||||
query_access_info.partitions.emplace(partition_name);
|
query_access_info.partitions.emplace(partition_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name)
|
||||||
|
{
|
||||||
|
if (!qualified_projection_name)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (isGlobalContext())
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info");
|
||||||
|
|
||||||
|
std::lock_guard<std::mutex> lock(query_access_info.mutex);
|
||||||
|
query_access_info.projections.emplace(fmt::format(
|
||||||
|
"{}.{}", qualified_projection_name.storage_id.getFullTableName(), backQuoteIfNeed(qualified_projection_name.projection_name)));
|
||||||
}
|
}
|
||||||
|
|
||||||
void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const
|
void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const
|
||||||
|
@ -658,6 +658,14 @@ public:
|
|||||||
const String & view_name = {});
|
const String & view_name = {});
|
||||||
void addQueryAccessInfo(const Names & partition_names);
|
void addQueryAccessInfo(const Names & partition_names);
|
||||||
|
|
||||||
|
struct QualifiedProjectionName
|
||||||
|
{
|
||||||
|
StorageID storage_id = StorageID::createEmpty();
|
||||||
|
String projection_name;
|
||||||
|
explicit operator bool() const { return !projection_name.empty(); }
|
||||||
|
};
|
||||||
|
void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name);
|
||||||
|
|
||||||
|
|
||||||
/// Supported factories for records in query_log
|
/// Supported factories for records in query_log
|
||||||
enum class QueryLogFactories
|
enum class QueryLogFactories
|
||||||
|
@ -337,6 +337,11 @@ std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & se
|
|||||||
LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name);
|
LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name);
|
||||||
return nullptr;
|
return nullptr;
|
||||||
}
|
}
|
||||||
|
if (dictionary->getSpecialKeyType() == DictionarySpecialKeyType::Range)
|
||||||
|
{
|
||||||
|
LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' is a range dictionary", dictionary_name);
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
auto dictionary_kv = std::dynamic_pointer_cast<const IKeyValueEntity>(dictionary);
|
auto dictionary_kv = std::dynamic_pointer_cast<const IKeyValueEntity>(dictionary);
|
||||||
table_join->setStorageJoin(dictionary_kv);
|
table_join->setStorageJoin(dictionary_kv);
|
||||||
|
@ -223,7 +223,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
|
|||||||
{
|
{
|
||||||
/// Set up memory profiling
|
/// Set up memory profiling
|
||||||
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
||||||
|
|
||||||
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
|
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
|
||||||
|
thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size);
|
||||||
|
thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size);
|
||||||
thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events);
|
thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -223,10 +223,10 @@ public:
|
|||||||
{
|
{
|
||||||
/// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm.
|
/// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm.
|
||||||
/// It's behaviour that was initially supported by clickhouse.
|
/// It's behaviour that was initially supported by clickhouse.
|
||||||
bool is_enbaled_by_default = val == JoinAlgorithm::DEFAULT
|
bool is_enabled_by_default = val == JoinAlgorithm::DEFAULT
|
||||||
|| val == JoinAlgorithm::HASH
|
|| val == JoinAlgorithm::HASH
|
||||||
|| val == JoinAlgorithm::DIRECT;
|
|| val == JoinAlgorithm::DIRECT;
|
||||||
if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enbaled_by_default)
|
if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enabled_by_default)
|
||||||
return true;
|
return true;
|
||||||
return join_algorithm.isSet(val);
|
return join_algorithm.isSet(val);
|
||||||
}
|
}
|
||||||
|
@ -83,6 +83,8 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex
|
|||||||
const Settings & settings = storage_context->getSettingsRef();
|
const Settings & settings = storage_context->getSettingsRef();
|
||||||
group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
|
||||||
group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
|
group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
|
||||||
|
group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size);
|
||||||
|
group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size);
|
||||||
group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator);
|
group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator);
|
||||||
group->memory_tracker.setParent(&background_memory_tracker);
|
group->memory_tracker.setParent(&background_memory_tracker);
|
||||||
if (settings.memory_tracker_fault_probability > 0.0)
|
if (settings.memory_tracker_fault_probability > 0.0)
|
||||||
|
@ -542,7 +542,8 @@ void trySetStorageInTableJoin(const QueryTreeNodePtr & table_expression, std::sh
|
|||||||
if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT))
|
if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
if (auto storage_dictionary = std::dynamic_pointer_cast<StorageDictionary>(storage); storage_dictionary)
|
if (auto storage_dictionary = std::dynamic_pointer_cast<StorageDictionary>(storage);
|
||||||
|
storage_dictionary && storage_dictionary->getDictionary()->getSpecialKeyType() != DictionarySpecialKeyType::Range)
|
||||||
table_join->setStorageJoin(std::dynamic_pointer_cast<const IKeyValueEntity>(storage_dictionary->getDictionary()));
|
table_join->setStorageJoin(std::dynamic_pointer_cast<const IKeyValueEntity>(storage_dictionary->getDictionary()));
|
||||||
else if (auto storage_key_value = std::dynamic_pointer_cast<IKeyValueEntity>(storage); storage_key_value)
|
else if (auto storage_key_value = std::dynamic_pointer_cast<IKeyValueEntity>(storage); storage_key_value)
|
||||||
table_join->setStorageJoin(storage_key_value);
|
table_join->setStorageJoin(storage_key_value);
|
||||||
|
@ -92,18 +92,6 @@ static AggregateProjectionInfo getAggregatingProjectionInfo(
|
|||||||
return info;
|
return info;
|
||||||
}
|
}
|
||||||
|
|
||||||
static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & names)
|
|
||||||
{
|
|
||||||
for (const auto & query_name : names)
|
|
||||||
{
|
|
||||||
auto jt = index.find(query_name);
|
|
||||||
if (jt == index.end() || jt->second->result_type->isNullable())
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
struct AggregateFunctionMatch
|
struct AggregateFunctionMatch
|
||||||
{
|
{
|
||||||
const AggregateDescription * description = nullptr;
|
const AggregateDescription * description = nullptr;
|
||||||
@ -170,21 +158,15 @@ std::optional<AggregateFunctionMatches> matchAggregateFunctions(
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// This is a special case for the function count().
|
/// This is a special case for the function count().
|
||||||
/// We can assume that 'count(expr) == count()' if expr is not nullable.
|
/// We can assume that 'count(expr) == count()' if expr is not nullable,
|
||||||
if (typeid_cast<const AggregateFunctionCount *>(candidate.function.get()))
|
/// which can be verified by simply casting to `AggregateFunctionCount *`.
|
||||||
{
|
if (typeid_cast<const AggregateFunctionCount *>(aggregate.function.get()))
|
||||||
bool has_nullable_or_missing_arg = false;
|
|
||||||
has_nullable_or_missing_arg |= hasNullableOrMissingColumn(query_index, aggregate.argument_names);
|
|
||||||
has_nullable_or_missing_arg |= hasNullableOrMissingColumn(proj_index, candidate.argument_names);
|
|
||||||
|
|
||||||
if (!has_nullable_or_missing_arg)
|
|
||||||
{
|
{
|
||||||
/// we can ignore arguments for count()
|
/// we can ignore arguments for count()
|
||||||
found_match = true;
|
found_match = true;
|
||||||
res.push_back({&candidate, DataTypes()});
|
res.push_back({&candidate, DataTypes()});
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
/// Now, function names and types matched.
|
/// Now, function names and types matched.
|
||||||
/// Next, match arguments from DAGs.
|
/// Next, match arguments from DAGs.
|
||||||
@ -628,8 +610,16 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
|
|||||||
// candidates.minmax_projection->block.dumpStructure());
|
// candidates.minmax_projection->block.dumpStructure());
|
||||||
|
|
||||||
Pipe pipe(std::make_shared<SourceFromSingleChunk>(std::move(candidates.minmax_projection->block)));
|
Pipe pipe(std::make_shared<SourceFromSingleChunk>(std::move(candidates.minmax_projection->block)));
|
||||||
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
|
projection_reading = std::make_unique<ReadFromPreparedSource>(
|
||||||
|
std::move(pipe),
|
||||||
|
context,
|
||||||
|
query_info.is_internal
|
||||||
|
? Context::QualifiedProjectionName{}
|
||||||
|
: Context::QualifiedProjectionName
|
||||||
|
{
|
||||||
|
.storage_id = reading->getMergeTreeData().getStorageID(),
|
||||||
|
.projection_name = candidates.minmax_projection->candidate.projection->name,
|
||||||
|
});
|
||||||
has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty();
|
has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty();
|
||||||
if (has_ordinary_parts)
|
if (has_ordinary_parts)
|
||||||
reading->resetParts(std::move(candidates.minmax_projection->normal_parts));
|
reading->resetParts(std::move(candidates.minmax_projection->normal_parts));
|
||||||
@ -661,7 +651,16 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
|
|||||||
{
|
{
|
||||||
auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames());
|
auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames());
|
||||||
Pipe pipe(std::make_shared<NullSource>(std::move(header)));
|
Pipe pipe(std::make_shared<NullSource>(std::move(header)));
|
||||||
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
|
projection_reading = std::make_unique<ReadFromPreparedSource>(
|
||||||
|
std::move(pipe),
|
||||||
|
context,
|
||||||
|
query_info.is_internal
|
||||||
|
? Context::QualifiedProjectionName{}
|
||||||
|
: Context::QualifiedProjectionName
|
||||||
|
{
|
||||||
|
.storage_id = reading->getMergeTreeData().getStorageID(),
|
||||||
|
.projection_name = best_candidate->projection->name,
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;
|
has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;
|
||||||
|
@ -187,7 +187,16 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
|
|||||||
if (!projection_reading)
|
if (!projection_reading)
|
||||||
{
|
{
|
||||||
Pipe pipe(std::make_shared<NullSource>(proj_snapshot->getSampleBlockForColumns(required_columns)));
|
Pipe pipe(std::make_shared<NullSource>(proj_snapshot->getSampleBlockForColumns(required_columns)));
|
||||||
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
|
projection_reading = std::make_unique<ReadFromPreparedSource>(
|
||||||
|
std::move(pipe),
|
||||||
|
context,
|
||||||
|
query_info.is_internal
|
||||||
|
? Context::QualifiedProjectionName{}
|
||||||
|
: Context::QualifiedProjectionName
|
||||||
|
{
|
||||||
|
.storage_id = reading->getMergeTreeData().getStorageID(),
|
||||||
|
.projection_name = best_candidate->projection->name,
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;
|
bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;
|
||||||
|
@ -1761,6 +1761,10 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
|||||||
fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id));
|
fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id));
|
||||||
}
|
}
|
||||||
context->getQueryContext()->addQueryAccessInfo(partition_names);
|
context->getQueryContext()->addQueryAccessInfo(partition_names);
|
||||||
|
|
||||||
|
if (storage_snapshot->projection)
|
||||||
|
context->getQueryContext()->addQueryAccessInfo(
|
||||||
|
Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name});
|
||||||
}
|
}
|
||||||
|
|
||||||
ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts);
|
ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts);
|
||||||
|
@ -4,14 +4,19 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_)
|
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, Context::QualifiedProjectionName qualified_projection_name_)
|
||||||
: ISourceStep(DataStream{.header = pipe_.getHeader()})
|
: ISourceStep(DataStream{.header = pipe_.getHeader()})
|
||||||
, pipe(std::move(pipe_))
|
, pipe(std::move(pipe_))
|
||||||
|
, context(std::move(context_))
|
||||||
|
, qualified_projection_name(std::move(qualified_projection_name_))
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||||
{
|
{
|
||||||
|
if (context && context->hasQueryContext())
|
||||||
|
context->getQueryContext()->addQueryAccessInfo(qualified_projection_name);
|
||||||
|
|
||||||
for (const auto & processor : pipe.getProcessors())
|
for (const auto & processor : pipe.getProcessors())
|
||||||
processors.emplace_back(processor);
|
processors.emplace_back(processor);
|
||||||
|
|
||||||
|
@ -1,4 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
#include <Processors/QueryPlan/ISourceStep.h>
|
#include <Processors/QueryPlan/ISourceStep.h>
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
|
|
||||||
@ -9,7 +11,8 @@ namespace DB
|
|||||||
class ReadFromPreparedSource : public ISourceStep
|
class ReadFromPreparedSource : public ISourceStep
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit ReadFromPreparedSource(Pipe pipe_);
|
explicit ReadFromPreparedSource(
|
||||||
|
Pipe pipe_, ContextPtr context_ = nullptr, Context::QualifiedProjectionName qualified_projection_name_ = {});
|
||||||
|
|
||||||
String getName() const override { return "ReadFromPreparedSource"; }
|
String getName() const override { return "ReadFromPreparedSource"; }
|
||||||
|
|
||||||
@ -18,6 +21,7 @@ public:
|
|||||||
protected:
|
protected:
|
||||||
Pipe pipe;
|
Pipe pipe;
|
||||||
ContextPtr context;
|
ContextPtr context;
|
||||||
|
Context::QualifiedProjectionName qualified_projection_name;
|
||||||
};
|
};
|
||||||
|
|
||||||
class ReadFromStorageStep : public ReadFromPreparedSource
|
class ReadFromStorageStep : public ReadFromPreparedSource
|
||||||
|
@ -89,7 +89,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory<S
|
|||||||
if (read_until_position < file_offset)
|
if (read_until_position < file_offset)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", file_offset, read_until_position - 1);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", file_offset, read_until_position - 1);
|
||||||
|
|
||||||
num_bytes_to_read = read_until_position - file_offset;
|
num_bytes_to_read = std::min<size_t>(read_until_position - file_offset, internal_buffer.size());
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
@ -130,3 +130,4 @@
|
|||||||
02581_share_big_sets_between_mutation_tasks_long
|
02581_share_big_sets_between_mutation_tasks_long
|
||||||
02581_share_big_sets_between_multiple_mutations_tasks_long
|
02581_share_big_sets_between_multiple_mutations_tasks_long
|
||||||
00992_system_parts_race_condition_zookeeper_long
|
00992_system_parts_race_condition_zookeeper_long
|
||||||
|
02815_range_dict_no_direct_join
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
#!/usr/bin/env python3
|
@ -0,0 +1,7 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<profiles>
|
||||||
|
<default>
|
||||||
|
<max_untracked_memory>1</max_untracked_memory>
|
||||||
|
</default>
|
||||||
|
</profiles>
|
||||||
|
</clickhouse>
|
@ -0,0 +1,5 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<total_memory_tracker_sample_probability>1</total_memory_tracker_sample_probability>
|
||||||
|
<total_memory_profiler_sample_min_allocation_size>4096</total_memory_profiler_sample_min_allocation_size>
|
||||||
|
<total_memory_profiler_sample_max_allocation_size>8192</total_memory_profiler_sample_max_allocation_size>
|
||||||
|
</clickhouse>
|
@ -0,0 +1,37 @@
|
|||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
import pytest
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
node = cluster.add_instance(
|
||||||
|
"node",
|
||||||
|
main_configs=["configs/memory_profiler.xml"],
|
||||||
|
user_configs=["configs/max_untracked_memory.xml"],
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
def test_trace_boundaries_work(started_cluster):
|
||||||
|
node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null")
|
||||||
|
node.query("SYSTEM FLUSH LOGS")
|
||||||
|
|
||||||
|
assert (
|
||||||
|
node.query(
|
||||||
|
"SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'"
|
||||||
|
)
|
||||||
|
== "1\n"
|
||||||
|
)
|
||||||
|
assert (
|
||||||
|
node.query(
|
||||||
|
"SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)"
|
||||||
|
)
|
||||||
|
== "0\n"
|
||||||
|
)
|
@ -24,8 +24,8 @@
|
|||||||
<value>'.*' || toString(number) || '.'</value>
|
<value>'.*' || toString(number) || '.'</value>
|
||||||
<!-- simple patterns, low distinctness (10 patterns) -->
|
<!-- simple patterns, low distinctness (10 patterns) -->
|
||||||
<value>'.*' || toString(number % 10) || '.'</value>
|
<value>'.*' || toString(number % 10) || '.'</value>
|
||||||
<!-- complex patterns, all unique -->
|
<!-- complex patterns, all unique - this is very slow (from 2 to 15 seconds) -->
|
||||||
<value>'([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number)</value>
|
<!-- <value>'([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number)</value> -->
|
||||||
<!-- complex patterns, low distinctness -->
|
<!-- complex patterns, low distinctness -->
|
||||||
<value>'([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10)</value>
|
<value>'([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10)</value>
|
||||||
<!-- Note: for this benchmark, we are only interested in compilation time, not correctness, evaluation time or the result.
|
<!-- Note: for this benchmark, we are only interested in compilation time, not correctness, evaluation time or the result.
|
||||||
|
@ -7,6 +7,11 @@ empty
|
|||||||
0
|
0
|
||||||
0
|
0
|
||||||
0
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
char
|
char
|
||||||
1
|
1
|
||||||
2
|
2
|
||||||
|
@ -12,6 +12,11 @@ select 'empty';
|
|||||||
select countSubstrings('', '.');
|
select countSubstrings('', '.');
|
||||||
select countSubstrings('', '');
|
select countSubstrings('', '');
|
||||||
select countSubstrings('.', '');
|
select countSubstrings('.', '');
|
||||||
|
select countSubstrings(toString(number), '') from numbers(1);
|
||||||
|
select countSubstrings('', toString(number)) from numbers(1);
|
||||||
|
select countSubstrings('aaa', materialize(''));
|
||||||
|
select countSubstrings(materialize('aaa'), '');
|
||||||
|
select countSubstrings(materialize('aaa'), materialize(''));
|
||||||
|
|
||||||
select 'char';
|
select 'char';
|
||||||
select countSubstrings('foobar.com', '.');
|
select countSubstrings('foobar.com', '.');
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
1
|
@ -0,0 +1,9 @@
|
|||||||
|
DROP TABLE IF EXISTS test;
|
||||||
|
|
||||||
|
CREATE TABLE test (`val` LowCardinality(Nullable(String))) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192;
|
||||||
|
|
||||||
|
insert into test select number == 3 ? 'some value' : null from numbers(5);
|
||||||
|
|
||||||
|
SELECT count(val) FROM test SETTINGS optimize_use_implicit_projections = 1;
|
||||||
|
|
||||||
|
DROP TABLE test;
|
@ -0,0 +1,3 @@
|
|||||||
|
t.t_normal
|
||||||
|
t.t_agg
|
||||||
|
t._minmax_count_projection
|
@ -0,0 +1,66 @@
|
|||||||
|
set log_queries=1;
|
||||||
|
set log_queries_min_type='QUERY_FINISH';
|
||||||
|
set optimize_use_implicit_projections=1;
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS t;
|
||||||
|
|
||||||
|
CREATE TABLE t
|
||||||
|
(
|
||||||
|
`id` UInt64,
|
||||||
|
`id2` UInt64,
|
||||||
|
`id3` UInt64,
|
||||||
|
PROJECTION t_normal
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
id,
|
||||||
|
id2,
|
||||||
|
id3
|
||||||
|
ORDER BY
|
||||||
|
id2,
|
||||||
|
id,
|
||||||
|
id3
|
||||||
|
),
|
||||||
|
PROJECTION t_agg
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
sum(id3)
|
||||||
|
GROUP BY id2
|
||||||
|
)
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree
|
||||||
|
ORDER BY id
|
||||||
|
SETTINGS index_granularity = 8;
|
||||||
|
|
||||||
|
insert into t SELECT number, -number, number FROM numbers(10000);
|
||||||
|
|
||||||
|
SELECT * FROM t WHERE id2 = 3 FORMAT Null;
|
||||||
|
SELECT sum(id3) FROM t GROUP BY id2 FORMAT Null;
|
||||||
|
SELECT min(id) FROM t FORMAT Null;
|
||||||
|
|
||||||
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
--Remove the prefix string which is a mutable database name.
|
||||||
|
arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.')
|
||||||
|
FROM
|
||||||
|
system.query_log
|
||||||
|
WHERE
|
||||||
|
current_database=currentDatabase() and query = 'SELECT * FROM t WHERE id2 = 3 FORMAT Null;';
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
--Remove the prefix string which is a mutable database name.
|
||||||
|
arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.')
|
||||||
|
FROM
|
||||||
|
system.query_log
|
||||||
|
WHERE
|
||||||
|
current_database=currentDatabase() and query = 'SELECT sum(id3) FROM t GROUP BY id2 FORMAT Null;';
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
--Remove the prefix string which is a mutable database name.
|
||||||
|
arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.')
|
||||||
|
FROM
|
||||||
|
system.query_log
|
||||||
|
WHERE
|
||||||
|
current_database=currentDatabase() and query = 'SELECT min(id) FROM t FORMAT Null;';
|
||||||
|
|
||||||
|
DROP TABLE t;
|
@ -1,15 +0,0 @@
|
|||||||
0 gY
|
|
||||||
1 jR
|
|
||||||
2 k5
|
|
||||||
3 l5
|
|
||||||
4 mO
|
|
||||||
0 pbgkmdljlpjoapne
|
|
||||||
1 akemglnjepjpodba
|
|
||||||
2 obmgndljgajpkeao
|
|
||||||
3 dldokmpjpgjgeanb
|
|
||||||
4 nkdlpgajngjnobme
|
|
||||||
YQrvD5XGvbx
|
|
||||||
Bm3zaOq7zbp
|
|
||||||
oV
|
|
||||||
oV
|
|
||||||
6b
|
|
@ -1,16 +0,0 @@
|
|||||||
-- Tags: no-upgrade-check
|
|
||||||
SET allow_experimental_hash_functions = 1;
|
|
||||||
|
|
||||||
select number, hashid(number) from system.numbers limit 5;
|
|
||||||
select number, hashid(number, 's3cr3t', 16, 'abcdefghijklmnop') from system.numbers limit 5;
|
|
||||||
select hashid(1234567890123456, 's3cr3t');
|
|
||||||
select hashid(1234567890123456, 's3cr3t2');
|
|
||||||
|
|
||||||
SELECT hashid(1, hashid(2));
|
|
||||||
SELECT hashid(1, 'k5');
|
|
||||||
SELECT hashid(1, 'k5_othersalt');
|
|
||||||
|
|
||||||
-- https://github.com/ClickHouse/ClickHouse/issues/39672
|
|
||||||
SELECT
|
|
||||||
JSONExtractRaw(257, NULL),
|
|
||||||
hashid(1024, if(rand() % 10, 'truetruetruetrue', NULL), 's3\0r3t'); -- {serverError 43}
|
|
@ -346,7 +346,6 @@ hasAny
|
|||||||
hasColumnInTable
|
hasColumnInTable
|
||||||
hasSubstr
|
hasSubstr
|
||||||
hasThreadFuzzer
|
hasThreadFuzzer
|
||||||
hashid
|
|
||||||
hex
|
hex
|
||||||
hiveHash
|
hiveHash
|
||||||
hop
|
hop
|
||||||
|
@ -197,40 +197,3 @@ E28DBDE7FE22E41C
|
|||||||
Check bug with hashing of const integer values
|
Check bug with hashing of const integer values
|
||||||
11862823756610506724
|
11862823756610506724
|
||||||
11862823756610506724
|
11862823756610506724
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
Check memsan bug
|
|
||||||
18096612095653370192
|
|
||||||
20AF99D3A87829E0
|
|
||||||
12489502208762728797
|
|
||||||
Check const columns
|
|
||||||
15080046610211022027
|
|
||||||
15080046610211022027
|
|
||||||
15080046610211022027
|
|
||||||
15080046610211022027
|
|
||||||
2E779C73D13981AA1AE19AFF9617EA49
|
|
||||||
2E779C73D13981AA1AE19AFF9617EA49
|
|
||||||
2E779C73D13981AA1AE19AFF9617EA49
|
|
||||||
2E779C73D13981AA1AE19AFF9617EA49
|
|
||||||
Check multiple keys as tuple from a table
|
|
||||||
11862823756610506724
|
|
||||||
9357996107237883963
|
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
F6D93D8FEA6D7DECCDD95A7A0A2AA36D
|
|
||||||
Check multiple keys as separate ints from a table
|
|
||||||
11862823756610506724
|
|
||||||
9357996107237883963
|
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
F6D93D8FEA6D7DECCDD95A7A0A2AA36D
|
|
||||||
Check constant key and data from a table
|
|
||||||
11862823756610506724
|
|
||||||
11862823756610506724
|
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
Check multiple keys as separate ints from a table with constant data
|
|
||||||
11862823756610506724
|
|
||||||
9357996107237883963
|
|
||||||
86AE90BB6A238D3F6221457630142C9B
|
|
||||||
F6D93D8FEA6D7DECCDD95A7A0A2AA36D
|
|
||||||
Check asan bug
|
|
||||||
0
|
|
||||||
|
@ -263,10 +263,10 @@ select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8,
|
|||||||
select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62));
|
select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62));
|
||||||
select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63));
|
select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63));
|
||||||
|
|
||||||
select sipHash64Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED }
|
select sipHash64Keyed((0, 0), '1'); -- { serverError 48 }
|
||||||
select sipHash128Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED }
|
select sipHash128Keyed((0, 0), '1'); -- { serverError 48 }
|
||||||
select sipHash64Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED }
|
select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 }
|
||||||
select sipHash128Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED }
|
select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 }
|
||||||
|
|
||||||
select hex(sipHash64());
|
select hex(sipHash64());
|
||||||
SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000';
|
SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000';
|
||||||
@ -280,57 +280,4 @@ INSERT INTO tab VALUES ((2, 2), 4);
|
|||||||
-- these two statements must produce the same result
|
-- these two statements must produce the same result
|
||||||
SELECT sipHash64Keyed(key, val) FROM tab;
|
SELECT sipHash64Keyed(key, val) FROM tab;
|
||||||
SELECT sipHash64Keyed(key, 4::UInt64) FROM tab;
|
SELECT sipHash64Keyed(key, 4::UInt64) FROM tab;
|
||||||
SELECT hex(sipHash128Keyed(key, val)) FROM tab;
|
|
||||||
SELECT hex(sipHash128Keyed(key, 4::UInt64)) FROM tab;
|
|
||||||
DROP TABLE tab;
|
DROP TABLE tab;
|
||||||
|
|
||||||
SELECT 'Check memsan bug';
|
|
||||||
SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2);
|
|
||||||
SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646));
|
|
||||||
SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806);
|
|
||||||
|
|
||||||
SELECT 'Check const columns';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_test;
|
|
||||||
CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b;
|
|
||||||
SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test');
|
|
||||||
SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test;
|
|
||||||
SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test;
|
|
||||||
SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test;
|
|
||||||
SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), 1, 'test'));
|
|
||||||
SELECT hex(sipHash128(tuple(*))) FROM sipHashKeyed_test;
|
|
||||||
SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test;
|
|
||||||
SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test;
|
|
||||||
DROP TABLE sipHashKeyed_test;
|
|
||||||
|
|
||||||
SELECT 'Check multiple keys as tuple from a table';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_keys;
|
|
||||||
CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4);
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4);
|
|
||||||
SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key;
|
|
||||||
SELECT hex(sipHash128Keyed(key, val)) FROM sipHashKeyed_keys ORDER by key;
|
|
||||||
DROP TABLE sipHashKeyed_keys;
|
|
||||||
|
|
||||||
SELECT 'Check multiple keys as separate ints from a table';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_keys;
|
|
||||||
CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4);
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4);
|
|
||||||
SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0;
|
|
||||||
SELECT hex(sipHash128Keyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0;
|
|
||||||
SELECT 'Check constant key and data from a table';
|
|
||||||
SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val;
|
|
||||||
SELECT hex(sipHash128Keyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val;
|
|
||||||
DROP TABLE sipHashKeyed_keys;
|
|
||||||
|
|
||||||
SELECT 'Check multiple keys as separate ints from a table with constant data';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_keys;
|
|
||||||
CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (2, 2);
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (4, 4);
|
|
||||||
SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0;
|
|
||||||
SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0;
|
|
||||||
DROP TABLE sipHashKeyed_keys;
|
|
||||||
|
|
||||||
SELECT 'Check asan bug';
|
|
||||||
SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806);
|
|
||||||
|
@ -1,152 +1 @@
|
|||||||
A3817F04BA25A8E66DF67214C7550293
|
|
||||||
DA87C1D86B99AF44347659119B22FC45
|
|
||||||
8177228DA4A45DC7FCA38BDEF60AFFE4
|
|
||||||
9C70B60C5267A94E5F33B6B02985ED51
|
|
||||||
F88164C12D9C8FAF7D0F6E7C7BCD5579
|
|
||||||
1368875980776F8854527A07690E9627
|
|
||||||
14EECA338B208613485EA0308FD7A15E
|
|
||||||
A1F1EBBED8DBC153C0B84AA61FF08239
|
|
||||||
3B62A9BA6258F5610F83E264F31497B4
|
|
||||||
264499060AD9BAABC47F8B02BB6D71ED
|
|
||||||
00110DC378146956C95447D3F3D0FBBA
|
|
||||||
0151C568386B6677A2B4DC6F81E5DC18
|
|
||||||
D626B266905EF35882634DF68532C125
|
|
||||||
9869E247E9C08B10D029934FC4B952F7
|
|
||||||
31FCEFAC66D7DE9C7EC7485FE4494902
|
|
||||||
5493E99933B0A8117E08EC0F97CFC3D9
|
|
||||||
6EE2A4CA67B054BBFD3315BF85230577
|
|
||||||
473D06E8738DB89854C066C47AE47740
|
|
||||||
A426E5E423BF4885294DA481FEAEF723
|
|
||||||
78017731CF65FAB074D5208952512EB1
|
|
||||||
9E25FC833F2290733E9344A5E83839EB
|
|
||||||
568E495ABE525A218A2214CD3E071D12
|
|
||||||
4A29B54552D16B9A469C10528EFF0AAE
|
|
||||||
C9D184DDD5A9F5E0CF8CE29A9ABF691C
|
|
||||||
2DB479AE78BD50D8882A8A178A6132AD
|
|
||||||
8ECE5F042D5E447B5051B9EACB8D8F6F
|
|
||||||
9C0B53B4B3C307E87EAEE08678141F66
|
|
||||||
ABF248AF69A6EAE4BFD3EB2F129EEB94
|
|
||||||
0664DA1668574B88B935F3027358AEF4
|
|
||||||
AA4B9DC4BF337DE90CD4FD3C467C6AB7
|
|
||||||
EA5C7F471FAF6BDE2B1AD7D4686D2287
|
|
||||||
2939B0183223FAFC1723DE4F52C43D35
|
|
||||||
7C3956CA5EEAFC3E363E9D556546EB68
|
|
||||||
77C6077146F01C32B6B69D5F4EA9FFCF
|
|
||||||
37A6986CB8847EDF0925F0F1309B54DE
|
|
||||||
A705F0E69DA9A8F907241A2E923C8CC8
|
|
||||||
3DC47D1F29C448461E9E76ED904F6711
|
|
||||||
0D62BF01E6FC0E1A0D3C4751C5D3692B
|
|
||||||
8C03468BCA7C669EE4FD5E084BBEE7B5
|
|
||||||
528A5BB93BAF2C9C4473CCE5D0D22BD9
|
|
||||||
DF6A301E95C95DAD97AE0CC8C6913BD8
|
|
||||||
801189902C857F39E73591285E70B6DB
|
|
||||||
E617346AC9C231BB3650AE34CCCA0C5B
|
|
||||||
27D93437EFB721AA401821DCEC5ADF89
|
|
||||||
89237D9DED9C5E78D8B1C9B166CC7342
|
|
||||||
4A6D8091BF5E7D651189FA94A250B14C
|
|
||||||
0E33F96055E7AE893FFC0E3DCF492902
|
|
||||||
E61C432B720B19D18EC8D84BDC63151B
|
|
||||||
F7E5AEF549F782CF379055A608269B16
|
|
||||||
438D030FD0B7A54FA837F2AD201A6403
|
|
||||||
A590D3EE4FBF04E3247E0D27F286423F
|
|
||||||
5FE2C1A172FE93C4B15CD37CAEF9F538
|
|
||||||
2C97325CBD06B36EB2133DD08B3A017C
|
|
||||||
92C814227A6BCA949FF0659F002AD39E
|
|
||||||
DCE850110BD8328CFBD50841D6911D87
|
|
||||||
67F14984C7DA791248E32BB5922583DA
|
|
||||||
1938F2CF72D54EE97E94166FA91D2A36
|
|
||||||
74481E9646ED49FE0F6224301604698E
|
|
||||||
57FCA5DE98A9D6D8006438D0583D8A1D
|
|
||||||
9FECDE1CEFDC1CBED4763674D9575359
|
|
||||||
E3040C00EB28F15366CA73CBD872E740
|
|
||||||
7697009A6A831DFECCA91C5993670F7A
|
|
||||||
5853542321F567A005D547A4F04759BD
|
|
||||||
5150D1772F50834A503E069A973FBD7C
|
|
||||||
1
|
1
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
1
|
|
||||||
Check bug with hashing of const integer values
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
Check memsan bug
|
|
||||||
1CE422FEE7BD8DE20000000000000000
|
|
||||||
Check const columns
|
|
||||||
B66B53476BDBEB8549A257E3B1766C30
|
|
||||||
B66B53476BDBEB8549A257E3B1766C30
|
|
||||||
B66B53476BDBEB8549A257E3B1766C30
|
|
||||||
B66B53476BDBEB8549A257E3B1766C30
|
|
||||||
Check multiple keys as tuple from a table
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
EC58946A98A0D37F4E3FAC02FBBA9480
|
|
||||||
Check multiple keys as separate ints from a table
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
EC58946A98A0D37F4E3FAC02FBBA9480
|
|
||||||
Check constant key and data from a table
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
Check multiple keys as separate ints from a table with constant data
|
|
||||||
E940B12600C844966162FF8FE7A16AAE
|
|
||||||
EC58946A98A0D37F4E3FAC02FBBA9480
|
|
||||||
|
@ -1,254 +1 @@
|
|||||||
-- Test Vectors from the SipHash reference C implementation:
|
|
||||||
-- Written by
|
|
||||||
-- Jean-Philippe Aumasson <jeanphilippe.aumasson@gmail.com>
|
|
||||||
-- Daniel J. Bernstein <djb@cr.yp.to>
|
|
||||||
-- Released under CC0
|
|
||||||
-- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645
|
|
||||||
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
''));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)));
|
|
||||||
select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)),
|
|
||||||
char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)));
|
|
||||||
|
|
||||||
-- CH tests
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62));
|
|
||||||
select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63));
|
|
||||||
|
|
||||||
select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED }
|
|
||||||
select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED }
|
|
||||||
|
|
||||||
SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000';
|
SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000';
|
||||||
SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000';
|
|
||||||
|
|
||||||
SELECT 'Check bug with hashing of const integer values';
|
|
||||||
DROP TABLE IF EXISTS tab;
|
|
||||||
CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO tab VALUES ((2, 2), 4);
|
|
||||||
-- these two statements must produce the same result
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM tab;
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed(key, 4::UInt64)) FROM tab;
|
|
||||||
DROP TABLE tab;
|
|
||||||
|
|
||||||
SELECT 'Check memsan bug';
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1);
|
|
||||||
|
|
||||||
SELECT 'Check const columns';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_test;
|
|
||||||
CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b;
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), 1, 'test'));
|
|
||||||
SELECT hex(sipHash128Reference(tuple(*))) FROM sipHashKeyed_test;
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test;
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test;
|
|
||||||
DROP TABLE sipHashKeyed_test;
|
|
||||||
|
|
||||||
SELECT 'Check multiple keys as tuple from a table';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_keys;
|
|
||||||
CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4);
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4);
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM sipHashKeyed_keys ORDER by key;
|
|
||||||
DROP TABLE sipHashKeyed_keys;
|
|
||||||
|
|
||||||
SELECT 'Check multiple keys as separate ints from a table';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_keys;
|
|
||||||
CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4);
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4);
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0;
|
|
||||||
SELECT 'Check constant key and data from a table';
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val;
|
|
||||||
DROP TABLE sipHashKeyed_keys;
|
|
||||||
|
|
||||||
SELECT 'Check multiple keys as separate ints from a table with constant data';
|
|
||||||
DROP TABLE IF EXISTS sipHashKeyed_keys;
|
|
||||||
CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory;
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (2, 2);
|
|
||||||
INSERT INTO sipHashKeyed_keys VALUES (4, 4);
|
|
||||||
SELECT hex(sipHash128ReferenceKeyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0;
|
|
||||||
DROP TABLE sipHashKeyed_keys;
|
|
||||||
|
@ -0,0 +1,12 @@
|
|||||||
|
1 0.1
|
||||||
|
1 0.2
|
||||||
|
2 0.3
|
||||||
|
2 0.4
|
||||||
|
3 0.5
|
||||||
|
3 0.6
|
||||||
|
1 0.1
|
||||||
|
1 0.2
|
||||||
|
2 0.3
|
||||||
|
2 0.4
|
||||||
|
3 0.5
|
||||||
|
3 0.6
|
@ -0,0 +1,34 @@
|
|||||||
|
CREATE TABLE discounts
|
||||||
|
(
|
||||||
|
advertiser_id UInt64,
|
||||||
|
discount_start_date Date,
|
||||||
|
discount_end_date Nullable(Date),
|
||||||
|
amount Float64
|
||||||
|
)
|
||||||
|
ENGINE = Memory;
|
||||||
|
|
||||||
|
INSERT INTO discounts VALUES (1, '2015-01-01', Null, 0.1);
|
||||||
|
INSERT INTO discounts VALUES (1, '2015-01-15', Null, 0.2);
|
||||||
|
INSERT INTO discounts VALUES (2, '2015-01-01', '2015-01-15', 0.3);
|
||||||
|
INSERT INTO discounts VALUES (2, '2015-01-04', '2015-01-10', 0.4);
|
||||||
|
INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-15', 0.5);
|
||||||
|
INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-10', 0.6);
|
||||||
|
|
||||||
|
CREATE DICTIONARY discounts_dict
|
||||||
|
(
|
||||||
|
advertiser_id UInt64,
|
||||||
|
discount_start_date Date,
|
||||||
|
discount_end_date Nullable(Date),
|
||||||
|
amount Float64
|
||||||
|
)
|
||||||
|
PRIMARY KEY advertiser_id
|
||||||
|
SOURCE(CLICKHOUSE(TABLE discounts))
|
||||||
|
LIFETIME(MIN 600 MAX 900)
|
||||||
|
LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'max'))
|
||||||
|
RANGE(MIN discount_start_date MAX discount_end_date);
|
||||||
|
|
||||||
|
CREATE TABLE ids (id UInt64) ENGINE = Memory;
|
||||||
|
INSERT INTO ids SELECT * FROM numbers(10);
|
||||||
|
|
||||||
|
SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS join_algorithm = 'direct';
|
||||||
|
SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS allow_experimental_analyzer = 1;
|
@ -0,0 +1 @@
|
|||||||
|
1
|
@ -0,0 +1,18 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings
|
||||||
|
# requires TraceCollector, does not available under sanitizers and aarch64
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CURDIR"/../shell_config.sh
|
||||||
|
|
||||||
|
query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM"
|
||||||
|
${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
|
||||||
|
|
||||||
|
# at least something allocated
|
||||||
|
${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'"
|
||||||
|
|
||||||
|
# show wrong allocations
|
||||||
|
${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)"
|
@ -0,0 +1 @@
|
|||||||
|
SELECT match('', repeat('(', 100000)); -- { serverError 306 }
|
2
tests/queries/0_stateless/02831_trash.reference
Normal file
2
tests/queries/0_stateless/02831_trash.reference
Normal file
@ -0,0 +1,2 @@
|
|||||||
|
2761631236
|
||||||
|
1210084689
|
2
tests/queries/0_stateless/02831_trash.sql
Normal file
2
tests/queries/0_stateless/02831_trash.sql
Normal file
@ -0,0 +1,2 @@
|
|||||||
|
SELECT CRC32IEEE(sipHash128());
|
||||||
|
SELECT CRC32(murmurHash3_128());
|
Loading…
Reference in New Issue
Block a user