Merge remote-tracking branch 'upstream/master' into fix27

This commit is contained in:
proller 2020-02-14 21:28:09 +03:00
commit cf39e9d64a
88 changed files with 2339 additions and 460 deletions

View File

@ -52,12 +52,12 @@ IncludeCategories:
ReflowComments: false
AlignEscapedNewlinesLeft: false
AlignEscapedNewlines: DontAlign
AlignTrailingComments: true
# Not changed:
AccessModifierOffset: -4
AlignConsecutiveAssignments: false
AlignOperands: false
AlignTrailingComments: false
AllowAllParametersOfDeclarationOnNextLine: true
AllowShortBlocksOnASingleLine: false
AllowShortCaseLabelsOnASingleLine: false

1
.gitignore vendored
View File

@ -15,6 +15,7 @@
/docs/build
/docs/publish
/docs/edit
/docs/website
/docs/tools/venv/
/docs/en/single.md
/docs/ru/single.md

View File

@ -1,13 +1,17 @@
[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.yandex)
[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.tech)
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
## Useful Links
* [Official website](https://clickhouse.yandex/) has quick high-level overview of ClickHouse on main page.
* [Tutorial](https://clickhouse.yandex/tutorial.html) shows how to set up and query small ClickHouse cluster.
* [Documentation](https://clickhouse.yandex/docs/en/) provides more in-depth information.
* [Official website](https://clickhouse.tech/) has quick high-level overview of ClickHouse on main page.
* [Tutorial](https://clickhouse.tech/docs/en/getting_started/tutorial/) shows how to set up and query small ClickHouse cluster.
* [Documentation](https://clickhouse.tech/docs/en/) provides more in-depth information.
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events.
* [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any.
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [ClickHouse Meetup in Athens](https://www.meetup.com/Athens-Big-Data/events/268379195/) on March 5.

2
contrib/avro vendored

@ -1 +1 @@
Subproject commit 5b2752041c8d2f75eb5c1dbec8b4c25fc0e24d12
Subproject commit 6cfcf6c24293af100d523b89b61d1ab216fa4735

View File

@ -1,12 +1,17 @@
#include <cstdlib>
#include "MemoryTracker.h"
#include <common/likely.h>
#include <common/logger_useful.h>
#include <IO/WriteHelpers.h>
#include "Common/TraceCollector.h"
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
#include <common/likely.h>
#include <common/logger_useful.h>
#include <ext/singleton.h>
#include <atomic>
#include <cmath>
#include <cstdlib>
namespace DB
@ -73,7 +78,7 @@ void MemoryTracker::alloc(Int64 size)
return;
/** Using memory_order_relaxed means that if allocations are done simultaneously,
* we allow exception about memory limit exceeded to be thrown only on next allocation.
* we allow exception about memory limit exceeded to be thrown only on next allocation.
* So, we allow over-allocations.
*/
Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed);
@ -81,7 +86,8 @@ void MemoryTracker::alloc(Int64 size)
if (metric != CurrentMetrics::end())
CurrentMetrics::add(metric, size);
Int64 current_limit = limit.load(std::memory_order_relaxed);
Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed);
Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed);
/// Using non-thread-safe random number generator. Joint distribution in different threads would not be uniform.
/// In this case, it doesn't matter.
@ -98,12 +104,19 @@ void MemoryTracker::alloc(Int64 size)
message << " " << description;
message << ": fault injected. Would use " << formatReadableSizeWithBinarySuffix(will_be)
<< " (attempt to allocate chunk of " << size << " bytes)"
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_limit);
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit);
throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED);
}
if (unlikely(current_limit && will_be > current_limit))
if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
{
auto no_track = blocker.cancel();
ext::Singleton<DB::TraceCollector>()->collect(size);
setOrRaiseProfilerLimit(current_profiler_limit + Int64(std::ceil((will_be - current_profiler_limit) / profiler_step)) * profiler_step);
}
if (unlikely(current_hard_limit && will_be > current_hard_limit))
{
free(size);
@ -116,7 +129,7 @@ void MemoryTracker::alloc(Int64 size)
message << " " << description;
message << " exceeded: would use " << formatReadableSizeWithBinarySuffix(will_be)
<< " (attempt to allocate chunk of " << size << " bytes)"
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_limit);
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit);
throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED);
}
@ -174,7 +187,8 @@ void MemoryTracker::resetCounters()
{
amount.store(0, std::memory_order_relaxed);
peak.store(0, std::memory_order_relaxed);
limit.store(0, std::memory_order_relaxed);
hard_limit.store(0, std::memory_order_relaxed);
profiler_limit.store(0, std::memory_order_relaxed);
}
@ -187,11 +201,20 @@ void MemoryTracker::reset()
}
void MemoryTracker::setOrRaiseLimit(Int64 value)
void MemoryTracker::setOrRaiseHardLimit(Int64 value)
{
/// This is just atomic set to maximum.
Int64 old_value = limit.load(std::memory_order_relaxed);
while (old_value < value && !limit.compare_exchange_weak(old_value, value))
Int64 old_value = hard_limit.load(std::memory_order_relaxed);
while (old_value < value && !hard_limit.compare_exchange_weak(old_value, value))
;
}
void MemoryTracker::setOrRaiseProfilerLimit(Int64 value)
{
/// This is just atomic set to maximum.
Int64 old_value = profiler_limit.load(std::memory_order_relaxed);
while (old_value < value && !profiler_limit.compare_exchange_weak(old_value, value))
;
}
@ -207,7 +230,7 @@ namespace CurrentMemoryTracker
if (untracked > untracked_memory_limit)
{
/// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes
/// more. It could be usefull for enlarge Exception message in rethrow logic.
/// more. It could be useful to enlarge Exception message in rethrow logic.
Int64 tmp = untracked;
untracked = 0;
memory_tracker->alloc(tmp);
@ -218,10 +241,7 @@ namespace CurrentMemoryTracker
void realloc(Int64 old_size, Int64 new_size)
{
Int64 addition = new_size - old_size;
if (addition > 0)
alloc(addition);
else
free(-addition);
addition > 0 ? alloc(addition) : free(-addition);
}
void free(Int64 size)

View File

@ -15,7 +15,10 @@ class MemoryTracker
{
std::atomic<Int64> amount {0};
std::atomic<Int64> peak {0};
std::atomic<Int64> limit {0};
std::atomic<Int64> hard_limit {0};
std::atomic<Int64> profiler_limit {0};
Int64 profiler_step = 0;
/// To test exception safety of calling code, memory tracker throws an exception on each memory allocation with specified probability.
double fault_probability = 0;
@ -32,7 +35,6 @@ class MemoryTracker
public:
MemoryTracker(VariableContext level_ = VariableContext::Thread) : level(level_) {}
MemoryTracker(Int64 limit_, VariableContext level_ = VariableContext::Thread) : limit(limit_), level(level_) {}
MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread) : parent(parent_), level(level_) {}
~MemoryTracker();
@ -66,21 +68,22 @@ public:
return peak.load(std::memory_order_relaxed);
}
void setLimit(Int64 limit_)
{
limit.store(limit_, std::memory_order_relaxed);
}
/** Set limit if it was not set.
* Otherwise, set limit to new value, if new value is greater than previous limit.
*/
void setOrRaiseLimit(Int64 value);
void setOrRaiseHardLimit(Int64 value);
void setOrRaiseProfilerLimit(Int64 value);
void setFaultProbability(double value)
{
fault_probability = value;
}
void setProfilerStep(Int64 value)
{
profiler_step = value;
}
/// next should be changed only once: from nullptr to some value.
/// NOTE: It is not true in MergeListElement
void setParent(MemoryTracker * elem)

View File

@ -1,92 +1,38 @@
#include "QueryProfiler.h"
#include <random>
#include <common/phdr_cache.h>
#include <common/config_common.h>
#include <common/StringRef.h>
#include <common/logger_useful.h>
#include <Common/PipeFDs.h>
#include <Common/StackTrace.h>
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/thread_local_rng.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
#include <Common/Exception.h>
#include <Common/StackTrace.h>
#include <Common/TraceCollector.h>
#include <Common/thread_local_rng.h>
#include <common/StringRef.h>
#include <common/config_common.h>
#include <common/logger_useful.h>
#include <common/phdr_cache.h>
#include <ext/singleton.h>
#include <random>
namespace ProfileEvents
{
extern const Event QueryProfilerSignalOverruns;
}
namespace DB
{
extern LazyPipeFDs trace_pipe;
namespace
{
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id.
/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler.
constexpr size_t QUERY_ID_MAX_LEN = 1024;
#if defined(OS_LINUX)
thread_local size_t write_trace_iteration = 0;
#endif
void writeTraceInfo(TimerType timer_type, int /* sig */, siginfo_t * info, void * context)
void writeTraceInfo(TraceType trace_type, int /* sig */, siginfo_t * info, void * context)
{
int overrun_count = 0;
#if defined(OS_LINUX)
/// Quickly drop if signal handler is called too frequently.
/// Otherwise we may end up infinitelly processing signals instead of doing any useful work.
++write_trace_iteration;
if (info && info->si_overrun > 0)
{
/// But pass with some frequency to avoid drop of all traces.
if (write_trace_iteration % info->si_overrun == 0)
{
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, info->si_overrun);
}
else
{
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, info->si_overrun + 1);
return;
}
}
if (info)
overrun_count = info->si_overrun;
#else
UNUSED(info);
#endif
constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag
8 * sizeof(char) + // maximum VarUInt length for string size
QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length
sizeof(UInt8) + // number of stack frames
sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity
sizeof(TimerType) + // timer type
sizeof(UInt64); // thread_id
char buffer[buf_size];
WriteBufferFromFileDescriptorDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);
StringRef query_id = CurrentThread::getQueryId();
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
UInt64 thread_id = CurrentThread::get().thread_id;
const auto signal_context = *reinterpret_cast<ucontext_t *>(context);
const StackTrace stack_trace(signal_context);
writeChar(false, out);
writeStringBinary(query_id, out);
size_t stack_trace_size = stack_trace.getSize();
size_t stack_trace_offset = stack_trace.getOffset();
writeIntBinary(UInt8(stack_trace_size - stack_trace_offset), out);
for (size_t i = stack_trace_offset; i < stack_trace_size; ++i)
writePODBinary(stack_trace.getFrames()[i], out);
writePODBinary(timer_type, out);
writePODBinary(thread_id, out);
out.next();
ext::Singleton<TraceCollector>()->collect(trace_type, stack_trace, overrun_count);
}
[[maybe_unused]] const UInt32 TIMER_PRECISION = 1e9;
@ -135,11 +81,11 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const UInt64 thread_id, const
sev.sigev_notify = SIGEV_THREAD_ID;
sev.sigev_signo = pause_signal;
#if defined(__FreeBSD__)
# if defined(__FreeBSD__)
sev._sigev_un._threadid = thread_id;
#else
# else
sev._sigev_un._tid = thread_id;
#endif
# endif
if (timer_create(clock_type, &sev, &timer_id))
{
/// In Google Cloud Run, the function "timer_create" is implemented incorrectly as of 2020-01-25.
@ -206,7 +152,7 @@ QueryProfilerReal::QueryProfilerReal(const UInt64 thread_id, const UInt32 period
void QueryProfilerReal::signalHandler(int sig, siginfo_t * info, void * context)
{
writeTraceInfo(TimerType::Real, sig, info, context);
writeTraceInfo(TraceType::REAL_TIME, sig, info, context);
}
QueryProfilerCpu::QueryProfilerCpu(const UInt64 thread_id, const UInt32 period)
@ -215,7 +161,7 @@ QueryProfilerCpu::QueryProfilerCpu(const UInt64 thread_id, const UInt32 period)
void QueryProfilerCpu::signalHandler(int sig, siginfo_t * info, void * context)
{
writeTraceInfo(TimerType::Cpu, sig, info, context);
writeTraceInfo(TraceType::CPU_TIME, sig, info, context);
}
}

View File

@ -15,12 +15,6 @@ namespace Poco
namespace DB
{
enum class TimerType : UInt8
{
Real,
Cpu,
};
/**
* Query profiler implementation for selected thread.
*

View File

@ -57,7 +57,7 @@ ShellCommand::~ShellCommand()
std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor)
{
/** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs,
* because of the resolving of characters in the shared library
* because of the resolving of symbols in the shared library
* http://www.oracle.com/technetwork/server-storage/solaris10/subprocess-136439.html
* Therefore, separate the resolving of the symbol from the call.
*/

View File

@ -1,25 +1,38 @@
#include "TraceCollector.h"
#include <Core/Field.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/TraceLog.h>
#include <Poco/Logger.h>
#include <Common/Exception.h>
#include <Common/PipeFDs.h>
#include <Common/StackTrace.h>
#include <common/logger_useful.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Common/Exception.h>
#include <Interpreters/TraceLog.h>
#include <unistd.h>
#include <fcntl.h>
namespace ProfileEvents
{
extern const Event QueryProfilerSignalOverruns;
}
namespace DB
{
LazyPipeFDs trace_pipe;
namespace
{
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id.
/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler.
constexpr size_t QUERY_ID_MAX_LEN = 1024;
thread_local size_t write_trace_iteration = 0;
}
namespace ErrorCodes
{
@ -27,20 +40,15 @@ namespace ErrorCodes
extern const int THREAD_IS_NOT_JOINABLE;
}
TraceCollector::TraceCollector(std::shared_ptr<TraceLog> & trace_log_)
: log(&Poco::Logger::get("TraceCollector"))
, trace_log(trace_log_)
TraceCollector::TraceCollector()
{
if (trace_log == nullptr)
throw Exception("Invalid trace log pointer passed", ErrorCodes::NULL_POINTER_DEREFERENCE);
trace_pipe.open();
pipe.open();
/** Turn write end of pipe to non-blocking mode to avoid deadlocks
* when QueryProfiler is invoked under locks and TraceCollector cannot pull data from pipe.
*/
trace_pipe.setNonBlocking();
trace_pipe.tryIncreaseSize(1 << 20);
pipe.setNonBlocking();
pipe.tryIncreaseSize(1 << 20);
thread = ThreadFromGlobalPool(&TraceCollector::run, this);
}
@ -48,14 +56,101 @@ TraceCollector::TraceCollector(std::shared_ptr<TraceLog> & trace_log_)
TraceCollector::~TraceCollector()
{
if (!thread.joinable())
LOG_ERROR(log, "TraceCollector thread is malformed and cannot be joined");
LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined");
else
{
TraceCollector::notifyToStop();
stop();
thread.join();
}
trace_pipe.close();
pipe.close();
}
void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trace, int overrun_count)
{
/// Quickly drop if signal handler is called too frequently.
/// Otherwise we may end up infinitelly processing signals instead of doing any useful work.
++write_trace_iteration;
if (overrun_count)
{
/// But pass with some frequency to avoid drop of all traces.
if (write_trace_iteration % overrun_count == 0)
{
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, overrun_count);
}
else
{
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, overrun_count + 1);
return;
}
}
constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag
8 * sizeof(char) + // maximum VarUInt length for string size
QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length
sizeof(UInt8) + // number of stack frames
sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity
sizeof(TraceType) + // trace type
sizeof(UInt64) + // thread_id
sizeof(UInt64); // size
char buffer[buf_size];
WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer);
StringRef query_id = CurrentThread::getQueryId();
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
auto thread_id = CurrentThread::get().thread_id;
writeChar(false, out);
writeStringBinary(query_id, out);
size_t stack_trace_size = stack_trace.getSize();
size_t stack_trace_offset = stack_trace.getOffset();
writeIntBinary(UInt8(stack_trace_size - stack_trace_offset), out);
for (size_t i = stack_trace_offset; i < stack_trace_size; ++i)
writePODBinary(stack_trace.getFrames()[i], out);
writePODBinary(trace_type, out);
writePODBinary(thread_id, out);
writePODBinary(UInt64(0), out);
out.next();
}
void TraceCollector::collect(UInt64 size)
{
constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag
8 * sizeof(char) + // maximum VarUInt length for string size
QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length
sizeof(UInt8) + // number of stack frames
sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity
sizeof(TraceType) + // trace type
sizeof(UInt64) + // thread_id
sizeof(UInt64); // size
char buffer[buf_size];
WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer);
StringRef query_id = CurrentThread::getQueryId();
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
auto thread_id = CurrentThread::get().thread_id;
writeChar(false, out);
writeStringBinary(query_id, out);
const auto & stack_trace = StackTrace();
size_t stack_trace_size = stack_trace.getSize();
size_t stack_trace_offset = stack_trace.getOffset();
writeIntBinary(UInt8(stack_trace_size - stack_trace_offset), out);
for (size_t i = stack_trace_offset; i < stack_trace_size; ++i)
writePODBinary(stack_trace.getFrames()[i], out);
writePODBinary(TraceType::MEMORY, out);
writePODBinary(thread_id, out);
writePODBinary(size, out);
out.next();
}
/**
@ -68,16 +163,16 @@ TraceCollector::~TraceCollector()
* NOTE: TraceCollector will NOT stop immediately as there may be some data left in the pipe
* before stop message.
*/
void TraceCollector::notifyToStop()
void TraceCollector::stop()
{
WriteBufferFromFileDescriptor out(trace_pipe.fds_rw[1]);
WriteBufferFromFileDescriptor out(pipe.fds_rw[1]);
writeChar(true, out);
out.next();
}
void TraceCollector::run()
{
ReadBufferFromFileDescriptor in(trace_pipe.fds_rw[0]);
ReadBufferFromFileDescriptor in(pipe.fds_rw[0]);
while (true)
{
@ -89,27 +184,33 @@ void TraceCollector::run()
std::string query_id;
readStringBinary(query_id, in);
UInt8 size = 0;
readIntBinary(size, in);
UInt8 trace_size = 0;
readIntBinary(trace_size, in);
Array trace;
trace.reserve(size);
trace.reserve(trace_size);
for (size_t i = 0; i < size; i++)
for (size_t i = 0; i < trace_size; i++)
{
uintptr_t addr = 0;
readPODBinary(addr, in);
trace.emplace_back(UInt64(addr));
}
TimerType timer_type;
readPODBinary(timer_type, in);
TraceType trace_type;
readPODBinary(trace_type, in);
UInt64 thread_id;
readPODBinary(thread_id, in);
TraceLogElement element{std::time(nullptr), timer_type, thread_id, query_id, trace};
trace_log->add(element);
UInt64 size;
readPODBinary(size, in);
if (trace_log)
{
TraceLogElement element{std::time(nullptr), trace_type, thread_id, query_id, trace, size};
trace_log->add(element);
}
}
}

View File

@ -1,7 +1,10 @@
#pragma once
#include "Common/PipeFDs.h"
#include <Common/ThreadPool.h>
class StackTrace;
namespace Poco
{
class Logger;
@ -12,21 +15,31 @@ namespace DB
class TraceLog;
enum class TraceType : UInt8
{
REAL_TIME,
CPU_TIME,
MEMORY,
};
class TraceCollector
{
public:
TraceCollector();
~TraceCollector();
void setTraceLog(const std::shared_ptr<TraceLog> & trace_log_) { trace_log = trace_log_; }
void collect(TraceType type, const StackTrace & stack_trace, int overrun_count = 0);
void collect(UInt64 size);
private:
Poco::Logger * log;
std::shared_ptr<TraceLog> trace_log;
ThreadFromGlobalPool thread;
LazyPipeFDs pipe;
void run();
static void notifyToStop();
public:
TraceCollector(std::shared_ptr<TraceLog> & trace_log_);
~TraceCollector();
void stop();
};
}

View File

@ -1,14 +1,16 @@
#if defined(OS_LINUX)
#include <malloc.h>
#elif defined(OS_DARWIN)
#include <malloc/malloc.h>
#endif
#include <new>
#include <common/config_common.h>
#include <common/memory.h>
#include <Common/MemoryTracker.h>
#include <iostream>
#include <new>
#if defined(OS_LINUX)
# include <malloc.h>
#elif defined(OS_DARWIN)
# include <malloc/malloc.h>
#endif
/// Replace default new/delete with memory tracking versions.
/// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new
/// https://en.cppreference.com/w/cpp/memory/new/operator_delete
@ -29,7 +31,7 @@ ALWAYS_INLINE void trackMemory(std::size_t size)
#endif
}
ALWAYS_INLINE bool trackMemoryNoExept(std::size_t size) noexcept
ALWAYS_INLINE bool trackMemoryNoExcept(std::size_t size) noexcept
{
try
{
@ -54,11 +56,11 @@ ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [
#else
if (size)
CurrentMemoryTracker::free(size);
#ifdef _GNU_SOURCE
# ifdef _GNU_SOURCE
/// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size.
else
CurrentMemoryTracker::free(malloc_usable_size(ptr));
#endif
# endif
#endif
}
catch (...)
@ -83,14 +85,14 @@ void * operator new[](std::size_t size)
void * operator new(std::size_t size, const std::nothrow_t &) noexcept
{
if (likely(Memory::trackMemoryNoExept(size)))
if (likely(Memory::trackMemoryNoExcept(size)))
return Memory::newNoExept(size);
return nullptr;
}
void * operator new[](std::size_t size, const std::nothrow_t &) noexcept
{
if (likely(Memory::trackMemoryNoExept(size)))
if (likely(Memory::trackMemoryNoExcept(size)))
return Memory::newNoExept(size);
return nullptr;
}

View File

@ -332,6 +332,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.", 0) \
M(SettingUInt64, memory_profiler_step, 0, "Every number of bytes the memory profiler will dump the allocating stacktrace. Zero means disabled memory profiler.", 0) \
\
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \

View File

@ -74,6 +74,11 @@ AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type)
return AttributeUnderlyingType::utDecimal128;
}
// Temporary hack to allow arrays in keys, since they are never retrieved for polygon dictionaries.
// TODO: This should be fixed by fully supporting arrays in dictionaries.
if (type.find("Array") == 0)
return AttributeUnderlyingType::utString;
throw Exception{"Unknown type " + type, ErrorCodes::UNKNOWN_TYPE};
}

View File

@ -0,0 +1,748 @@
#include <ext/map.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include "PolygonDictionary.h"
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
#include <numeric>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
}
IPolygonDictionary::IPolygonDictionary(
const std::string & database_,
const std::string & name_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
, dict_struct(dict_struct_)
, source_ptr(std::move(source_ptr_))
, dict_lifetime(dict_lifetime_)
, input_type(input_type_)
, point_type(point_type_)
{
createAttributes();
loadData();
}
const std::string & IPolygonDictionary::getDatabase() const
{
return database;
}
const std::string & IPolygonDictionary::getName() const
{
return name;
}
const std::string & IPolygonDictionary::getFullName() const
{
return full_name;
}
std::string IPolygonDictionary::getTypeName() const
{
return "Polygon";
}
std::string IPolygonDictionary::getKeyDescription() const
{
return dict_struct.getKeyDescription();
}
size_t IPolygonDictionary::getBytesAllocated() const
{
return bytes_allocated;
}
size_t IPolygonDictionary::getQueryCount() const
{
return query_count.load(std::memory_order_relaxed);
}
double IPolygonDictionary::getHitRate() const
{
return 1.0;
}
size_t IPolygonDictionary::getElementCount() const
{
return element_count;
}
double IPolygonDictionary::getLoadFactor() const
{
return 1.0;
}
const IDictionarySource * IPolygonDictionary::getSource() const
{
return source_ptr.get();
}
const DictionaryLifetime & IPolygonDictionary::getLifetime() const
{
return dict_lifetime;
}
const DictionaryStructure & IPolygonDictionary::getStructure() const
{
return dict_struct;
}
bool IPolygonDictionary::isInjective(const std::string &) const
{
return false;
}
BlockInputStreamPtr IPolygonDictionary::getBlockInputStream(const Names &, size_t) const
{
// TODO: In order for this to work one would first have to support retrieving arrays from dictionaries.
// I believe this is a separate task done by some other people.
throw Exception{"Reading the dictionary is not allowed", ErrorCodes::UNSUPPORTED_METHOD};
}
template <typename T>
void IPolygonDictionary::appendNullValueImpl(const Field & null_value)
{
null_values.emplace_back(T(null_value.get<NearestFieldType<T>>()));
}
void IPolygonDictionary::appendNullValue(AttributeUnderlyingType type, const Field & null_value)
{
switch (type)
{
case AttributeUnderlyingType::utUInt8:
appendNullValueImpl<UInt8>(null_value);
break;
case AttributeUnderlyingType::utUInt16:
appendNullValueImpl<UInt16>(null_value);
break;
case AttributeUnderlyingType::utUInt32:
appendNullValueImpl<UInt32>(null_value);
break;
case AttributeUnderlyingType::utUInt64:
appendNullValueImpl<UInt64>(null_value);
break;
case AttributeUnderlyingType::utUInt128:
appendNullValueImpl<UInt128>(null_value);
break;
case AttributeUnderlyingType::utInt8:
appendNullValueImpl<Int8>(null_value);
break;
case AttributeUnderlyingType::utInt16:
appendNullValueImpl<Int16>(null_value);
break;
case AttributeUnderlyingType::utInt32:
appendNullValueImpl<Int32>(null_value);
break;
case AttributeUnderlyingType::utInt64:
appendNullValueImpl<Int64>(null_value);
break;
case AttributeUnderlyingType::utFloat32:
appendNullValueImpl<Float32>(null_value);
break;
case AttributeUnderlyingType::utFloat64:
appendNullValueImpl<Float64>(null_value);
break;
case AttributeUnderlyingType::utDecimal32:
appendNullValueImpl<Decimal32>(null_value);
break;
case AttributeUnderlyingType::utDecimal64:
appendNullValueImpl<Decimal64>(null_value);
break;
case AttributeUnderlyingType::utDecimal128:
appendNullValueImpl<Decimal128>(null_value);
break;
case AttributeUnderlyingType::utString:
appendNullValueImpl<String>(null_value);
break;
}
}
void IPolygonDictionary::createAttributes()
{
attributes.resize(dict_struct.attributes.size());
for (size_t i = 0; i < dict_struct.attributes.size(); ++i)
{
const auto & attr = dict_struct.attributes[i];
attribute_index_by_name.emplace(attr.name, i);
appendNullValue(attr.underlying_type, attr.null_value);
if (attr.hierarchical)
throw Exception{name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH};
}
}
void IPolygonDictionary::blockToAttributes(const DB::Block &block)
{
const auto rows = block.rows();
element_count += rows;
for (size_t i = 0; i < attributes.size(); ++i)
{
const auto & column = block.safeGetByPosition(i + 1);
if (attributes[i])
{
MutableColumnPtr mutated = std::move(*attributes[i]).mutate();
mutated->insertRangeFrom(*column.column, 0, column.column->size());
attributes[i] = std::move(mutated);
}
else
attributes[i] = column.column;
}
/** Multi-polygons could cause bigger sizes, but this is better than nothing. */
polygons.reserve(polygons.size() + rows);
ids.reserve(ids.size() + rows);
const auto & key = block.safeGetByPosition(0).column;
extractPolygons(key);
}
void IPolygonDictionary::loadData()
{
auto stream = source_ptr->loadAll();
stream->readPrefix();
while (const auto block = stream->read())
blockToAttributes(block);
stream->readSuffix();
for (auto & polygon : polygons)
bg::correct(polygon);
}
void IPolygonDictionary::calculateBytesAllocated()
{
// TODO:: Account for key.
for (const auto & column : attributes)
bytes_allocated += column->allocatedBytes();
}
std::vector<IPolygonDictionary::Point> IPolygonDictionary::extractPoints(const Columns &key_columns)
{
if (key_columns.size() != 2)
throw Exception{"Expected two columns of coordinates", ErrorCodes::BAD_ARGUMENTS};
const auto column_x = typeid_cast<const ColumnVector<Float64>*>(key_columns[0].get());
const auto column_y = typeid_cast<const ColumnVector<Float64>*>(key_columns[1].get());
if (!column_x || !column_y)
throw Exception{"Expected columns of Float64", ErrorCodes::TYPE_MISMATCH};
const auto rows = key_columns.front()->size();
std::vector<Point> result;
result.reserve(rows);
for (const auto row : ext::range(0, rows))
result.emplace_back(column_x->getElement(row), column_y->getElement(row));
return result;
}
void IPolygonDictionary::has(const Columns &key_columns, const DataTypes &, PaddedPODArray<UInt8> &out) const
{
size_t row = 0;
for (const auto & pt : extractPoints(key_columns))
{
size_t trash = 0;
out[row] = find(pt, trash);
++row;
}
query_count.fetch_add(row, std::memory_order_relaxed);
}
size_t IPolygonDictionary::getAttributeIndex(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == attribute_index_by_name.end())
throw Exception{"No such attribute: " + attribute_name, ErrorCodes::BAD_ARGUMENTS};
return it->second;
}
#define DECLARE(TYPE) \
void IPolygonDictionary::get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ResultArrayType<TYPE> & out) const \
{ \
const auto ind = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(null_values[ind]); \
\
getItemsImpl<TYPE, TYPE>( \
ind, \
key_columns, \
[&](const size_t row, const auto value) { out[row] = value; }, \
[&](const size_t) { return null_value; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(UInt128)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
DECLARE(Decimal32)
DECLARE(Decimal64)
DECLARE(Decimal128)
#undef DECLARE
void IPolygonDictionary::getString(
const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ColumnString * out) const
{
const auto ind = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
const auto & null_value = StringRef{std::get<String>(null_values[ind])};
getItemsImpl<String, StringRef>(
ind,
key_columns,
[&](const size_t, const StringRef & value) { out->insertData(value.data, value.size); },
[&](const size_t) { return null_value; });
}
#define DECLARE(TYPE) \
void IPolygonDictionary::get##TYPE( \
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes &, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const \
{ \
const auto ind = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
ind, \
key_columns, \
[&](const size_t row, const auto value) { out[row] = value; }, \
[&](const size_t row) { return def[row]; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(UInt128)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
DECLARE(Decimal32)
DECLARE(Decimal64)
DECLARE(Decimal128)
#undef DECLARE
void IPolygonDictionary::getString(
const std::string & attribute_name,
const Columns & key_columns,
const DataTypes &,
const ColumnString * const def,
ColumnString * const out) const
{
const auto ind = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
getItemsImpl<String, StringRef>(
ind,
key_columns,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t row) { return def->getDataAt(row); });
}
#define DECLARE(TYPE) \
void IPolygonDictionary::get##TYPE( \
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes &, \
const TYPE def, \
ResultArrayType<TYPE> & out) const \
{ \
const auto ind = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
ind, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(UInt128)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
DECLARE(Decimal32)
DECLARE(Decimal64)
DECLARE(Decimal128)
#undef DECLARE
void IPolygonDictionary::getString(
const std::string & attribute_name,
const Columns & key_columns,
const DataTypes &,
const String & def,
ColumnString * const out) const
{
const auto ind = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
getItemsImpl<String, StringRef>(
ind,
key_columns,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return StringRef{def}; });
}
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void IPolygonDictionary::getItemsImpl(
size_t attribute_ind, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const
{
const auto points = extractPoints(key_columns);
using ColVecType = std::conditional_t<IsDecimalNumber<AttributeType>, ColumnDecimal<AttributeType>, ColumnVector<AttributeType>>;
using ColType = std::conditional_t<std::is_same<AttributeType, String>::value, ColumnString, ColVecType>;
const auto column = typeid_cast<const ColType *>(attributes[attribute_ind].get());
if (!column)
throw Exception{"An attribute should be a column of its type", ErrorCodes::BAD_ARGUMENTS};
for (const auto i : ext::range(0, points.size()))
{
size_t id = 0;
const auto found = find(points[i], id);
id = ids[id];
if (!found)
{
set_value(i, static_cast<OutputType>(get_default(i)));
continue;
}
if constexpr (std::is_same<AttributeType, String>::value)
set_value(i, static_cast<OutputType>(column->getDataAt(id)));
else
set_value(i, static_cast<OutputType>(column->getElement(id)));
}
query_count.fetch_add(points.size(), std::memory_order_relaxed);
}
namespace
{
struct Offset
{
Offset() = default;
IColumn::Offsets ring_offsets;
IColumn::Offsets polygon_offsets;
IColumn::Offsets multi_polygon_offsets;
IColumn::Offset points_added = 0;
IColumn::Offset current_ring = 0;
IColumn::Offset current_polygon = 0;
IColumn::Offset current_multi_polygon = 0;
Offset& operator++()
{
++points_added;
if (points_added <= ring_offsets[current_ring])
return *this;
++current_ring;
if (current_ring < polygon_offsets[current_polygon])
return *this;
++current_polygon;
if (current_polygon < multi_polygon_offsets[current_multi_polygon])
return *this;
++current_multi_polygon;
return *this;
}
bool atLastPolygonOfMultiPolygon() { return current_polygon + 1 == multi_polygon_offsets[current_multi_polygon]; }
bool atLastRingOfPolygon() { return current_ring + 1 == polygon_offsets[current_polygon]; }
bool atLastPointOfRing() { return points_added == ring_offsets[current_ring]; }
bool allRingsHaveAPositiveArea()
{
IColumn::Offset prev_offset = 0;
for (const auto offset : ring_offsets)
{
if (offset - prev_offset < 3)
return false;
prev_offset = offset;
}
return true;
}
};
struct Data
{
std::vector<IPolygonDictionary::Polygon> & dest;
std::vector<size_t> & ids;
void addPolygon(bool new_multi_polygon = false)
{
dest.emplace_back();
ids.push_back((ids.empty() ? 0 : ids.back() + new_multi_polygon));
}
void addPoint(Float64 x, Float64 y)
{
auto & last_polygon = dest.back();
auto & last_ring = (last_polygon.inners().empty() ? last_polygon.outer() : last_polygon.inners().back());
last_ring.emplace_back(x, y);
}
};
void addNewPoint(Float64 x, Float64 y, Data & data, Offset & offset)
{
if (offset.atLastPointOfRing())
{
if (offset.atLastRingOfPolygon())
data.addPolygon(offset.atLastPolygonOfMultiPolygon());
else
{
/** An outer ring is added automatically with a new polygon, thus we need the else statement here.
* This also implies that if we are at this point we have to add an inner ring.
*/
auto & last_polygon = data.dest.back();
last_polygon.inners().emplace_back();
}
}
data.addPoint(x, y);
++offset;
}
const IColumn * unrollMultiPolygons(const ColumnPtr & column, Offset & offset)
{
const auto ptr_multi_polygons = typeid_cast<const ColumnArray*>(column.get());
if (!ptr_multi_polygons)
throw Exception{"Expected a column containing arrays of polygons", ErrorCodes::TYPE_MISMATCH};
offset.multi_polygon_offsets.assign(ptr_multi_polygons->getOffsets());
const auto ptr_polygons = typeid_cast<const ColumnArray*>(&ptr_multi_polygons->getData());
if (!ptr_polygons)
throw Exception{"Expected a column containing arrays of rings when reading polygons", ErrorCodes::TYPE_MISMATCH};
offset.polygon_offsets.assign(ptr_polygons->getOffsets());
const auto ptr_rings = typeid_cast<const ColumnArray*>(&ptr_polygons->getData());
if (!ptr_rings)
throw Exception{"Expected a column containing arrays of points when reading rings", ErrorCodes::TYPE_MISMATCH};
offset.ring_offsets.assign(ptr_rings->getOffsets());
return ptr_rings->getDataPtr().get();
}
const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset)
{
const auto ptr_polygons = typeid_cast<const ColumnArray*>(column.get());
if (!ptr_polygons)
throw Exception{"Expected a column containing arrays of points", ErrorCodes::TYPE_MISMATCH};
offset.ring_offsets.assign(ptr_polygons->getOffsets());
std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1);
offset.multi_polygon_offsets.assign(offset.polygon_offsets);
return ptr_polygons->getDataPtr().get();
}
void handlePointsReprByArrays(const IColumn * column, Data & data, Offset & offset)
{
const auto ptr_points = typeid_cast<const ColumnArray*>(column);
const auto ptr_coord = typeid_cast<const ColumnVector<Float64>*>(&ptr_points->getData());
if (!ptr_coord)
throw Exception{"Expected coordinates to be of type Float64", ErrorCodes::TYPE_MISMATCH};
const auto & offsets = ptr_points->getOffsets();
IColumn::Offset prev_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
if (offsets[i] - prev_offset != 2)
throw Exception{"All points should be two-dimensional", ErrorCodes::BAD_ARGUMENTS};
prev_offset = offsets[i];
addNewPoint(ptr_coord->getElement(2 * i), ptr_coord->getElement(2 * i + 1), data, offset);
}
}
void handlePointsReprByTuples(const IColumn * column, Data & data, Offset & offset)
{
const auto ptr_points = typeid_cast<const ColumnTuple*>(column);
if (!ptr_points)
throw Exception{"Expected a column of tuples representing points", ErrorCodes::TYPE_MISMATCH};
if (ptr_points->tupleSize() != 2)
throw Exception{"Points should be two-dimensional", ErrorCodes::BAD_ARGUMENTS};
const auto column_x = typeid_cast<const ColumnVector<Float64>*>(&ptr_points->getColumn(0));
const auto column_y = typeid_cast<const ColumnVector<Float64>*>(&ptr_points->getColumn(1));
if (!column_x || !column_y)
throw Exception{"Expected coordinates to be of type Float64", ErrorCodes::TYPE_MISMATCH};
for (size_t i = 0; i < column_x->size(); ++i)
{
addNewPoint(column_x->getElement(i), column_y->getElement(i), data, offset);
}
}
}
void IPolygonDictionary::extractPolygons(const ColumnPtr &column)
{
Data data = {polygons, ids};
Offset offset;
const IColumn * points_collection = nullptr;
switch (input_type)
{
case InputType::MultiPolygon:
points_collection = unrollMultiPolygons(column, offset);
break;
case InputType::SimplePolygon:
points_collection = unrollSimplePolygons(column, offset);
break;
}
if (!offset.allRingsHaveAPositiveArea())
throw Exception{"Every ring included in a polygon or excluded from it should contain at least 3 points",
ErrorCodes::BAD_ARGUMENTS};
/** Adding the first empty polygon */
data.addPolygon(true);
switch (point_type)
{
case PointType::Array:
handlePointsReprByArrays(points_collection, data, offset);
break;
case PointType::Tuple:
handlePointsReprByTuples(points_collection, data, offset);
break;
}
}
SimplePolygonDictionary::SimplePolygonDictionary(
const std::string & database_,
const std::string & name_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_)
: IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_)
{
}
std::shared_ptr<const IExternalLoadable> SimplePolygonDictionary::clone() const
{
return std::make_shared<SimplePolygonDictionary>(
this->database,
this->name,
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
this->input_type,
this->point_type);
}
bool SimplePolygonDictionary::find(const Point &point, size_t & id) const
{
bool found = false;
double area = 0;
for (size_t i = 0; i < (this->polygons).size(); ++i)
{
if (bg::covered_by(point, (this->polygons)[i]))
{
double new_area = bg::area((this->polygons)[i]);
if (!found || new_area < area)
{
found = true;
id = i;
area = new_area;
}
}
}
return found;
}
void registerDictionaryPolygon(DictionaryFactory & factory)
{
auto create_layout = [=](const std::string &,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
if (!dict_struct.key)
throw Exception{"'key' is required for a dictionary of layout 'polygon'", ErrorCodes::BAD_ARGUMENTS};
if (dict_struct.key->size() != 1)
throw Exception{"The 'key' should consist of a single attribute for a dictionary of layout 'polygon'",
ErrorCodes::BAD_ARGUMENTS};
IPolygonDictionary::InputType input_type;
IPolygonDictionary::PointType point_type;
const auto key_type = (*dict_struct.key)[0].type;
const auto f64 = std::make_shared<DataTypeFloat64>();
const auto multi_polygon_array = DataTypeArray(std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(f64))));
const auto multi_polygon_tuple = DataTypeArray(std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(std::vector<DataTypePtr>{f64, f64}))));
const auto simple_polygon_array = DataTypeArray(std::make_shared<DataTypeArray>(f64));
const auto simple_polygon_tuple = DataTypeArray(std::make_shared<DataTypeTuple>(std::vector<DataTypePtr>{f64, f64}));
if (key_type->equals(multi_polygon_array))
{
input_type = IPolygonDictionary::InputType::MultiPolygon;
point_type = IPolygonDictionary::PointType::Array;
}
else if (key_type->equals(multi_polygon_tuple))
{
input_type = IPolygonDictionary::InputType::MultiPolygon;
point_type = IPolygonDictionary::PointType::Tuple;
}
else if (key_type->equals(simple_polygon_array))
{
input_type = IPolygonDictionary::InputType::SimplePolygon;
point_type = IPolygonDictionary::PointType::Array;
}
else if (key_type->equals(simple_polygon_tuple))
{
input_type = IPolygonDictionary::InputType::SimplePolygon;
point_type = IPolygonDictionary::PointType::Tuple;
}
else
throw Exception{"The key type " + key_type->getName() +
" is not one of the following allowed types for a dictionary of layout 'polygon': " +
multi_polygon_array.getName() + " " +
multi_polygon_tuple.getName() + " " +
simple_polygon_array.getName() + " " +
simple_polygon_tuple.getName() + " ",
ErrorCodes::BAD_ARGUMENTS};
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{name
+ ": elements range_min and range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<SimplePolygonDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type);
};
factory.registerLayout("polygon", create_layout, true);
}
}

View File

@ -0,0 +1,293 @@
#pragma once
#include <atomic>
#include <variant>
#include <Core/Block.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <boost/geometry.hpp>
#include <boost/geometry/geometries/multi_polygon.hpp>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
namespace DB
{
namespace bg = boost::geometry;
/** An interface for polygon dictionaries.
* Polygons are read and stored as multi_polygons from boost::geometry in Euclidean coordinates.
* An implementation should inherit from this base class and preprocess the data upon construction if needed.
* It must override the find method of this class which retrieves the polygon containing a single point.
*/
class IPolygonDictionary : public IDictionaryBase
{
public:
/** Controls the different types of polygons allowed as input.
* The structure of a multi-polygon is as follows:
* - A multi-polygon is represented by a nonempty array of polygons.
* - A polygon is represented by a nonempty array of rings. The first element represents the outer ring. Zero
* or more following rings are cut out from the polygon.
* - A ring is represented by a nonempty array of points.
* - A point is represented by its coordinates stored in an according structure (see below).
* A simple polygon is represented by an one-dimensional array of points, stored in the according structure.
*/
enum class InputType
{
MultiPolygon,
SimplePolygon
};
/** Controls the different types allowed for providing the coordinates of points.
* Right now a point can be represented by either an array or a tuple of two Float64 values.
*/
enum class PointType
{
Array,
Tuple,
};
IPolygonDictionary(
const std::string & database_,
const std::string & name_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_);
const std::string & getDatabase() const override;
const std::string & getName() const override;
const std::string & getFullName() const override;
std::string getTypeName() const override;
std::string getKeyDescription() const;
size_t getBytesAllocated() const override;
size_t getQueryCount() const override;
double getHitRate() const override;
size_t getElementCount() const override;
double getLoadFactor() const override;
const IDictionarySource * getSource() const override;
const DictionaryStructure & getStructure() const override;
const DictionaryLifetime & getLifetime() const override;
bool isInjective(const std::string & attribute_name) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
template <typename T>
using ResultArrayType = std::conditional_t<IsDecimalNumber<T>, DecimalPaddedPODArray<T>, PaddedPODArray<T>>;
/** Functions used to retrieve attributes of specific type by key. */
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(UInt128)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
DECLARE(Decimal32)
DECLARE(Decimal64)
DECLARE(Decimal128)
#undef DECLARE
void getString(const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ColumnString * out) const;
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes &, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(UInt128)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
DECLARE(Decimal32)
DECLARE(Decimal64)
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name,
const Columns & key_columns,
const DataTypes &,
const ColumnString * const def,
ColumnString * const out) const;
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes &, \
const TYPE def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(UInt128)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
DECLARE(Decimal32)
DECLARE(Decimal64)
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name,
const Columns & key_columns,
const DataTypes & key_types,
const String & def,
ColumnString * const out) const;
/** Checks whether or not a point can be found in one of the polygons in the dictionary.
* The check is performed for multiple points represented by columns of their x and y coordinates.
* The boolean result is written to out.
*/
// TODO: Refactor the whole dictionary design to perform stronger checks, i.e. make this an override.
void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray<UInt8> & out) const;
/** A two-dimensional point in Euclidean coordinates. */
using Point = bg::model::point<Float64, 2, bg::cs::cartesian>;
/** A polygon in boost is a an outer ring of points with zero or more cut out inner rings. */
using Polygon = bg::model::polygon<Point>;
protected:
/** Returns true if the given point can be found in the polygon dictionary.
* If true id is set to the index of a polygon containing the given point.
* Overridden in different implementations of this interface.
*/
virtual bool find(const Point & point, size_t & id) const = 0;
std::vector<Polygon> polygons;
/** Since the original data may have been in the form of multi-polygons, an id is stored for each single polygon
* corresponding to the row in which any other attributes for this entry are located.
*/
std::vector<size_t> ids;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const InputType input_type;
const PointType point_type;
private:
/** Helper functions for loading the data from the configuration.
* The polygons serving as keys are extracted into boost types.
* All other values are stored in one column per attribute.
*/
void createAttributes();
void blockToAttributes(const Block & block);
void loadData();
void calculateBytesAllocated();
/** Checks whether a given attribute exists and returns its index */
size_t getAttributeIndex(const std::string & attribute_name) const;
/** Helper functions to retrieve and instantiate the provided null value of an attribute.
* Since a null value is obligatory for every attribute they are simply appended to null_values defined below.
*/
template <typename T>
void appendNullValueImpl(const Field & null_value);
void appendNullValue(AttributeUnderlyingType type, const Field & value);
/** Helper function for retrieving the value of an attribute by key. */
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsImpl(size_t attribute_ind, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const;
/** A mapping from the names of the attributes to their index in the two vectors defined below. */
std::map<std::string, size_t> attribute_index_by_name;
/** A vector of columns storing the values of each attribute. */
Columns attributes;
/** A vector of null values corresponding to each attribute. */
std::vector<std::variant<
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
String>> null_values;
size_t bytes_allocated = 0;
size_t element_count = 0;
mutable std::atomic<size_t> query_count{0};
/** Extracts a list of polygons from a column according to input_type and point_type.
* The polygons are appended to the dictionary with the corresponding ids.
*/
void extractPolygons(const ColumnPtr & column);
/** Extracts a list of points from two columns representing their x and y coordinates. */
static std::vector<Point> extractPoints(const Columns &key_columns);
};
/** Simple implementation of the polygon dictionary. Doesn't generate anything during its construction.
* Iterates over all stored polygons for each query, checking each of them in linear time.
* Retrieves the polygon with the smallest area containing the given point. If there is more than one any such polygon
* may be returned.
*/
class SimplePolygonDictionary : public IPolygonDictionary
{
public:
SimplePolygonDictionary(
const std::string & database_,
const std::string & name_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_);
std::shared_ptr<const IExternalLoadable> clone() const override;
private:
bool find(const Point & point, size_t & id) const override;
};
}

View File

@ -29,6 +29,7 @@ void registerDictionaries()
registerDictionaryFlat(factory);
registerDictionaryHashed(factory);
registerDictionaryCache(factory);
registerDictionaryPolygon(factory);
}
}

View File

@ -24,6 +24,7 @@ void registerDictionaryTrie(DictionaryFactory & factory);
void registerDictionaryFlat(DictionaryFactory & factory);
void registerDictionaryHashed(DictionaryFactory & factory);
void registerDictionaryCache(DictionaryFactory & factory);
void registerDictionaryPolygon(DictionaryFactory & factory);
void registerDictionaries();
}

View File

@ -33,6 +33,7 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h>
#include <Dictionaries/RangeHashedDictionary.h>
#include <Dictionaries/TrieDictionary.h>
#include <Dictionaries/PolygonDictionary.h>
#include <ext/range.h>
@ -134,7 +135,8 @@ private:
!executeDispatchSimple<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
@ -305,6 +307,7 @@ private:
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
@ -485,6 +488,7 @@ private:
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
@ -823,6 +827,7 @@ private:
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
@ -1081,6 +1086,7 @@ private:
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}

View File

@ -1,3 +1,4 @@
#include "Common/quoteString.h"
#include <Common/typeid_cast.h>
#include <Common/PODArray.h>
#include <Core/Row.h>
@ -334,7 +335,7 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast,
found = true;
if (found)
throw Exception("Column " + column_name.get(ast) + " is not under aggregate function and not in GROUP BY.",
throw Exception("Column " + backQuote(column_name.get(ast)) + " is not under aggregate function and not in GROUP BY",
ErrorCodes::NOT_AN_AGGREGATE);
/// Special check for WITH statement alias. Add alias action to be able to use this alias.

View File

@ -57,6 +57,7 @@
#include <Common/TraceCollector.h>
#include <common/logger_useful.h>
#include <Common/RemoteHostFilter.h>
#include <ext/singleton.h>
namespace ProfileEvents
{
@ -168,7 +169,6 @@ struct ContextShared
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
std::unique_ptr<TraceCollector> trace_collector; /// Thread collecting traces from threads executing queries
/// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests.
class SessionKeyHash
@ -299,13 +299,7 @@ struct ContextShared
schedule_pool.reset();
ddl_worker.reset();
/// Stop trace collector if any
trace_collector.reset();
}
bool hasTraceCollector()
{
return trace_collector != nullptr;
ext::Singleton<TraceCollector>::reset();
}
void initializeTraceCollector(std::shared_ptr<TraceLog> trace_log)
@ -313,7 +307,7 @@ struct ContextShared
if (trace_log == nullptr)
return;
trace_collector = std::make_unique<TraceCollector>(trace_log);
ext::Singleton<TraceCollector>()->setTraceLog(trace_log);
}
};
@ -1693,11 +1687,6 @@ void Context::initializeSystemLogs()
shared->system_logs.emplace(*global_context, getConfigRef());
}
bool Context::hasTraceCollector()
{
return shared->hasTraceCollector();
}
void Context::initializeTraceCollector()
{
shared->initializeTraceCollector(getTraceLog());

View File

@ -59,6 +59,8 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
if (!query.like.empty())
rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << "LIKE " << std::quoted(query.like, '\'');
else if (query.where_expression)
rewritten_query << " AND (" << query.where_expression << ")";
if (query.limit_length)
rewritten_query << " LIMIT " << query.limit_length;

View File

@ -181,12 +181,12 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
/// You should specify this value in configuration for default profile,
/// not for specific users, sessions or queries,
/// because this setting is effectively global.
total_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_all_queries);
total_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_all_queries);
total_memory_tracker.setDescription("(total)");
/// Track memory usage for all simultaneously running queries from single user.
user_process_list.user_memory_tracker.setParent(&total_memory_tracker);
user_process_list.user_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setDescription("(for user)");
/// Actualize thread group info
@ -198,7 +198,9 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
thread_group->query = process_it->query;
/// Set query-level memory trackers
thread_group->memory_tracker.setOrRaiseLimit(process_it->max_memory_usage);
thread_group->memory_tracker.setOrRaiseHardLimit(process_it->max_memory_usage);
thread_group->memory_tracker.setOrRaiseProfilerLimit(settings.memory_profiler_step);
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
thread_group->memory_tracker.setDescription("(for query)");
if (process_it->memory_tracker_fault_probability)
thread_group->memory_tracker.setFaultProbability(process_it->memory_tracker_fault_probability);

View File

@ -1,12 +1,15 @@
#include <Common/ThreadStatus.h>
#include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/QueryThreadLog.h>
#include <Common/CurrentThread.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/Exception.h>
#include <Common/QueryProfiler.h>
#include <Interpreters/Context.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/ProcessList.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/TraceCollector.h>
#include <ext/singleton.h>
#if defined(OS_LINUX)
# include <Common/hasLinuxCapability.h>
@ -153,7 +156,7 @@ void ThreadStatus::finalizePerformanceCounters()
void ThreadStatus::initQueryProfiler()
{
/// query profilers are useless without trace collector
if (!global_context || !global_context->hasTraceCollector())
if (!global_context || !ext::Singleton<TraceCollector>::isInitialized())
return;
const auto & settings = query_context->getSettingsRef();

View File

@ -9,11 +9,12 @@
using namespace DB;
using TimerDataType = TraceLogElement::TimerDataType;
using TraceDataType = TraceLogElement::TraceDataType;
const TimerDataType::Values TraceLogElement::timer_values = {
{"Real", static_cast<UInt8>(TimerType::Real)},
{"CPU", static_cast<UInt8>(TimerType::Cpu)}
const TraceDataType::Values TraceLogElement::trace_values = {
{"Real", static_cast<UInt8>(TraceType::REAL_TIME)},
{"CPU", static_cast<UInt8>(TraceType::CPU_TIME)},
{"Memory", static_cast<UInt8>(TraceType::MEMORY)},
};
Block TraceLogElement::createBlock()
@ -23,10 +24,11 @@ Block TraceLogElement::createBlock()
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeUInt32>(), "revision"},
{std::make_shared<TimerDataType>(timer_values), "timer_type"},
{std::make_shared<TraceDataType>(trace_values), "trace_type"},
{std::make_shared<DataTypeUInt64>(), "thread_id"},
{std::make_shared<DataTypeString>(), "query_id"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "trace"}
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "trace"},
{std::make_shared<DataTypeUInt64>(), "size"},
};
}
@ -39,10 +41,11 @@ void TraceLogElement::appendToBlock(Block & block) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(ClickHouseRevision::get());
columns[i++]->insert(static_cast<UInt8>(timer_type));
columns[i++]->insert(static_cast<UInt8>(trace_type));
columns[i++]->insert(thread_id);
columns[i++]->insertData(query_id.data(), query_id.size());
columns[i++]->insert(trace);
columns[i++]->insert(size);
block.setColumns(std::move(columns));
}

View File

@ -1,23 +1,26 @@
#pragma once
#include <Common/QueryProfiler.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/SystemLog.h>
#include <Common/QueryProfiler.h>
#include <Common/TraceCollector.h>
namespace DB
{
struct TraceLogElement
{
using TimerDataType = DataTypeEnum8;
static const TimerDataType::Values timer_values;
using TraceDataType = DataTypeEnum8;
static const TraceDataType::Values trace_values;
time_t event_time{};
TimerType timer_type{};
UInt64 thread_id{};
String query_id{};
Array trace{};
time_t event_time;
TraceType trace_type;
UInt64 thread_id;
String query_id;
Array trace;
UInt64 size; /// Allocation size in bytes for |TraceType::MEMORY|
static std::string name() { return "TraceLog"; }
static Block createBlock();

View File

@ -32,6 +32,11 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
if (!like.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << (not_like ? " NOT" : "") << " LIKE " << (settings.hilite ? hilite_none : "")
<< std::quoted(like, '\'');
else if (where_expression)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
where_expression->formatImpl(settings, state, frame);
}
if (limit_length)
{

View File

@ -20,6 +20,7 @@ public:
String from;
String like;
bool not_like{false};
ASTPtr where_expression;
ASTPtr limit_length;
/** Get the text that identifies this element. */

View File

@ -22,12 +22,14 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserKeyword s_databases("DATABASES");
ParserKeyword s_dictionaries("DICTIONARIES");
ParserKeyword s_from("FROM");
ParserKeyword s_in("IN");
ParserKeyword s_not("NOT");
ParserKeyword s_like("LIKE");
ParserKeyword s_where("WHERE");
ParserKeyword s_limit("LIMIT");
ParserStringLiteral like_p;
ParserIdentifier name_p;
ParserExpressionWithOptionalAlias limit_p(false);
ParserExpressionWithOptionalAlias exp_elem(false);
ASTPtr like;
ASTPtr database;
@ -54,7 +56,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
if (s_from.ignore(pos, expected))
if (s_from.ignore(pos, expected) || s_in.ignore(pos, expected))
{
if (!name_p.parse(pos, database, expected))
return false;
@ -70,10 +72,15 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
else if (query->not_like)
return false;
else if (s_where.ignore(pos, expected))
{
if (!exp_elem.parse(pos, query->where_expression, expected))
return false;
}
if (s_limit.ignore(pos, expected))
{
if (!limit_p.parse(pos, query->limit_length, expected))
if (!exp_elem.parse(pos, query->limit_length, expected))
return false;
}
}

View File

@ -333,8 +333,8 @@ void MergeTreeRangeReader::ReadResult::optimize()
filter_holder_original = std::move(filter_holder);
filter = new_filter.get();
filter_holder = std::move(new_filter);
need_filter = true;
}
need_filter = true;
}
/// Another guess, if it's worth filtering at PREWHERE
else if (countBytesInResultFilter(filter->getData()) < 0.6 * filter->size())

View File

@ -0,0 +1,65 @@
<test>
<type>once</type>
<stop_conditions>
<all_of>
<total_time_ms>30000</total_time_ms>
</all_of>
</stop_conditions>
<settings>
<max_threads>1</max_threads>
</settings>
<substitutions>
<substitution>
<name>aggregationscale</name>
<values>
<value>11111111</value>
<value>11111</value>
</values>
</substitution>
</substitutions>
<create_query>
create table mingroupby_orderbylimit1_{aggregationscale}_tuple
Engine=MergeTree order by tuple() AS
select toUInt64( number % {aggregationscale} ) key, toUInt64(1) value
from numbers(10000000);
</create_query>
<create_query>
create table mingroupby_orderbylimit1_{aggregationscale}_key_value
Engine=MergeTree order by (key,value) AS
select toUInt64( number % {aggregationscale} ) key, toUInt64(1) value
from numbers(10000000);
</create_query>
<query tag='UsingGroupbyMinTuple'>
SELECT key, min(value)
FROM mingroupby_orderbylimit1_{aggregationscale}_tuple
group by key format Null;
</query>
<query tag='UsingGroupbyMinKV'>
SELECT key, min(value)
FROM mingroupby_orderbylimit1_{aggregationscale}_key_value
group by key format Null;
</query>
<query tag='UsingOrderbyLimit1Tuple'>
SELECT key, value
FROM mingroupby_orderbylimit1_{aggregationscale}_tuple
order by key, value limit 1 by key format Null;
</query>
<query tag='UsingOrderbyLimit1KV'>
SELECT key, value
FROM mingroupby_orderbylimit1_{aggregationscale}_key_value
order by key, value limit 1 by key format Null;
</query>
<drop_query> DROP TABLE IF EXISTS mingroupby_orderbylimit1_{aggregationscale}_tuple </drop_query>
<drop_query> DROP TABLE IF EXISTS mingroupby_orderbylimit1_{aggregationscale}_key_value </drop_query>
</test>

View File

@ -1,5 +1,7 @@
A
B
numbers
one
A 1 TinyLog CREATE TABLE test_show_tables.A (`A` UInt8) ENGINE = TinyLog
B 1 TinyLog CREATE TABLE test_show_tables.B (`A` UInt8) ENGINE = TinyLog
test_temporary_table

View File

@ -6,6 +6,7 @@ CREATE TABLE test_show_tables.A (A UInt8) ENGINE = TinyLog;
CREATE TABLE test_show_tables.B (A UInt8) ENGINE = TinyLog;
SHOW TABLES from test_show_tables;
SHOW TABLES in system where engine like '%System%' and name in ('numbers', 'one');
SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = 'test_show_tables' ORDER BY name FORMAT TSVRaw;

View File

@ -0,0 +1,118 @@
dictGet test_01037.dict_array (-100,-42) qqq 101
dictGet test_01037.dict_array (-1,0) Click South 423
dictGet test_01037.dict_array (-0.1,0) Click South 423
dictGet test_01037.dict_array (0,-2) Click West 424
dictGet test_01037.dict_array (0,-1.1) Click West 424
dictGet test_01037.dict_array (0,1.1) Click North 422
dictGet test_01037.dict_array (0,2) Click North 422
dictGet test_01037.dict_array (0.1,0) Click East 421
dictGet test_01037.dict_array (0.99,2.99) Click North 422
dictGet test_01037.dict_array (1,0) Click East 421
dictGet test_01037.dict_array (3,3) House 314159
dictGet test_01037.dict_array (5,6) Click 42
dictGet test_01037.dict_array (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_array (-100,-42) www 1234
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (3,3) House 314159
dictGetOrDefault test_01037.dict_array (5,6) Click 42
dictGetOrDefault test_01037.dict_array (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_array (-100,-42) dd 44
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (3,3) House 314159
dictGetOrDefault test_01037.dict_array (5,6) Click 42
dictGetOrDefault test_01037.dict_array (7.01,7.01) ee 55
dictGet test_01037.dict_tuple (-100,-42) qqq 101
dictGet test_01037.dict_tuple (-1,0) Click South 423
dictGet test_01037.dict_tuple (-0.1,0) Click South 423
dictGet test_01037.dict_tuple (0,-2) Click West 424
dictGet test_01037.dict_tuple (0,-1.1) Click West 424
dictGet test_01037.dict_tuple (0,1.1) Click North 422
dictGet test_01037.dict_tuple (0,2) Click North 422
dictGet test_01037.dict_tuple (0.1,0) Click East 421
dictGet test_01037.dict_tuple (0.99,2.99) Click North 422
dictGet test_01037.dict_tuple (1,0) Click East 421
dictGet test_01037.dict_tuple (3,3) House 314159
dictGet test_01037.dict_tuple (5,6) Click 42
dictGet test_01037.dict_tuple (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_tuple (-100,-42) www 1234
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (3,3) House 314159
dictGetOrDefault test_01037.dict_tuple (5,6) Click 42
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_tuple (-100,-42) dd 44
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (3,3) House 314159
dictGetOrDefault test_01037.dict_tuple (5,6) Click 42
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) ee 55
dictHas test_01037.dict_array (-100,-42) 0
dictHas test_01037.dict_array (-1,0) 1
dictHas test_01037.dict_array (-0.1,0) 1
dictHas test_01037.dict_array (0,-2) 1
dictHas test_01037.dict_array (0,-1.1) 1
dictHas test_01037.dict_array (0,-1) 1
dictHas test_01037.dict_array (0,0) 1
dictHas test_01037.dict_array (0,1) 1
dictHas test_01037.dict_array (0,1.1) 1
dictHas test_01037.dict_array (0,2) 1
dictHas test_01037.dict_array (0.1,0) 1
dictHas test_01037.dict_array (0.99,2.99) 1
dictHas test_01037.dict_array (1,0) 1
dictHas test_01037.dict_array (1,1) 1
dictHas test_01037.dict_array (1,3) 1
dictHas test_01037.dict_array (3,3) 1
dictHas test_01037.dict_array (5,1) 1
dictHas test_01037.dict_array (5,5) 1
dictHas test_01037.dict_array (5,6) 1
dictHas test_01037.dict_array (7.01,7.01) 0
dictHas test_01037.dict_tuple (-100,-42) 0
dictHas test_01037.dict_tuple (-1,0) 1
dictHas test_01037.dict_tuple (-0.1,0) 1
dictHas test_01037.dict_tuple (0,-2) 1
dictHas test_01037.dict_tuple (0,-1.1) 1
dictHas test_01037.dict_tuple (0,-1) 1
dictHas test_01037.dict_tuple (0,0) 1
dictHas test_01037.dict_tuple (0,1) 1
dictHas test_01037.dict_tuple (0,1.1) 1
dictHas test_01037.dict_tuple (0,2) 1
dictHas test_01037.dict_tuple (0.1,0) 1
dictHas test_01037.dict_tuple (0.99,2.99) 1
dictHas test_01037.dict_tuple (1,0) 1
dictHas test_01037.dict_tuple (1,1) 1
dictHas test_01037.dict_tuple (1,3) 1
dictHas test_01037.dict_tuple (3,3) 1
dictHas test_01037.dict_tuple (5,1) 1
dictHas test_01037.dict_tuple (5,5) 1
dictHas test_01037.dict_tuple (5,6) 1
dictHas test_01037.dict_tuple (7.01,7.01) 0

View File

@ -0,0 +1,107 @@
SET send_logs_level = 'none';
DROP DATABASE IF EXISTS test_01037;
CREATE DATABASE test_01037 Engine = Ordinary;
DROP DICTIONARY IF EXISTS test_01037.dict_array;
DROP TABLE IF EXISTS test_01037.polygons_array;
CREATE TABLE test_01037.polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_array VALUES ([[[[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]], [[[5, 5], [5, 1], [7, 1], [7, 7], [1, 7], [1, 5]]]], 'Click', 42);
INSERT INTO test_01037.polygons_array VALUES ([[[[5, 5], [5, -5], [-5, -5], [-5, 5]], [[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]]], 'House', 314159);
INSERT INTO test_01037.polygons_array VALUES ([[[[3, 1], [0, 1], [0, -1], [3, -1]]]], 'Click East', 421);
INSERT INTO test_01037.polygons_array VALUES ([[[[-1, 1], [1, 1], [1, 3], [-1, 3]]]], 'Click North', 422);
INSERT INTO test_01037.polygons_array VALUES ([[[[-3, 1], [-3, -1], [0, -1], [0, 1]]]], 'Click South', 423);
INSERT INTO test_01037.polygons_array VALUES ([[[[-1, -1], [1, -1], [1, -3], [-1, -3]]]], 'Click West', 424);
CREATE DICTIONARY test_01037.dict_array
(
key Array(Array(Array(Array(Float64)))),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'polygons_array' PASSWORD '' DB 'test_01037'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(POLYGON());
DROP DICTIONARY IF EXISTS test_01037.dict_tuple;
DROP TABLE IF EXISTS test_01037.polygons_tuple;
CREATE TABLE test_01037.polygons_tuple (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_tuple VALUES ([[[(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]], [[(5, 5), (5, 1), (7, 1), (7, 7), (1, 7), (1, 5)]]], 'Click', 42);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(5, 5), (5, -5), (-5, -5), (-5, 5)], [(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]]], 'House', 314159);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424);
CREATE DICTIONARY test_01037.dict_tuple
(
key Array(Array(Array(Tuple(Float64, Float64)))),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'polygons_tuple' PASSWORD '' DB 'test_01037'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(POLYGON());
DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
INSERT INTO test_01037.points VALUES (0.1, 0.0, 112, 'aax');
INSERT INTO test_01037.points VALUES (-0.1, 0.0, 113, 'aay');
INSERT INTO test_01037.points VALUES (0.0, 1.1, 114, 'aaz');
INSERT INTO test_01037.points VALUES (0.0, -1.1, 115, 'aat');
INSERT INTO test_01037.points VALUES (3.0, 3.0, 22, 'bb');
INSERT INTO test_01037.points VALUES (5.0, 6.0, 33, 'cc');
INSERT INTO test_01037.points VALUES (-100.0, -42.0, 44, 'dd');
INSERT INTO test_01037.points VALUES (7.01, 7.01, 55, 'ee')
INSERT INTO test_01037.points VALUES (0.99, 2.99, 66, 'ee');
INSERT INTO test_01037.points VALUES (1.0, 0.0, 771, 'ffa');
INSERT INTO test_01037.points VALUES (-1.0, 0.0, 772, 'ffb');
INSERT INTO test_01037.points VALUES (0.0, 2.0, 773, 'ffc');
INSERT INTO test_01037.points VALUES (0.0, -2.0, 774, 'ffd');
select 'dictGet', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
select 'dictGet', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
INSERT INTO test_01037.points VALUES (5.0, 5.0, 0, '');
INSERT INTO test_01037.points VALUES (5.0, 1.0, 0, '');
INSERT INTO test_01037.points VALUES (1.0, 3.0, 0, '');
INSERT INTO test_01037.points VALUES (0.0, 0.0, 0, '');
INSERT INTO test_01037.points VALUES (0.0, 1.0, 0, '');
INSERT INTO test_01037.points VALUES (0.0, -1.0, 0, '');
INSERT INTO test_01037.points VALUES (1.0, 1.0, 0, '');
select 'dictHas', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
select 'dictHas', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
DROP DICTIONARY test_01037.dict_array;
DROP DICTIONARY test_01037.dict_tuple;
DROP TABLE test_01037.polygons_array;
DROP TABLE test_01037.polygons_tuple;
DROP TABLE test_01037.points;
DROP DATABASE test_01037;

View File

@ -0,0 +1,142 @@
dictGet test_01037.dict_array (-100,-42) qqq 101
dictGet test_01037.dict_array (-1,0) Click South 423
dictGet test_01037.dict_array (-0.1,0) Click South 423
dictGet test_01037.dict_array (0,-2) Click West 424
dictGet test_01037.dict_array (0,-1.1) Click West 424
dictGet test_01037.dict_array (0,1.1) Click North 422
dictGet test_01037.dict_array (0,2) Click North 422
dictGet test_01037.dict_array (0.1,0) Click East 421
dictGet test_01037.dict_array (0.99,2.99) Click North 422
dictGet test_01037.dict_array (1,0) Click East 421
dictGet test_01037.dict_array (2,4) House 523
dictGet test_01037.dict_array (2,4.1) qqq 101
dictGet test_01037.dict_array (3,3) House 523
dictGet test_01037.dict_array (4,4) House 523
dictGet test_01037.dict_array (5,6) qqq 101
dictGet test_01037.dict_array (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_array (-100,-42) www 1234
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (2,4) House 523
dictGetOrDefault test_01037.dict_array (2,4.1) www 1234
dictGetOrDefault test_01037.dict_array (3,3) House 523
dictGetOrDefault test_01037.dict_array (4,4) House 523
dictGetOrDefault test_01037.dict_array (5,6) www 1234
dictGetOrDefault test_01037.dict_array (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_array (-100,-42) dd 44
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (2,4) House 523
dictGetOrDefault test_01037.dict_array (2,4.1) gac 803
dictGetOrDefault test_01037.dict_array (3,3) House 523
dictGetOrDefault test_01037.dict_array (4,4) House 523
dictGetOrDefault test_01037.dict_array (5,6) cc 33
dictGetOrDefault test_01037.dict_array (7.01,7.01) ee 55
dictGet test_01037.dict_tuple (-100,-42) qqq 101
dictGet test_01037.dict_tuple (-1,0) Click South 423
dictGet test_01037.dict_tuple (-0.1,0) Click South 423
dictGet test_01037.dict_tuple (0,-2) Click West 424
dictGet test_01037.dict_tuple (0,-1.1) Click West 424
dictGet test_01037.dict_tuple (0,1.1) Click North 422
dictGet test_01037.dict_tuple (0,2) Click North 422
dictGet test_01037.dict_tuple (0.1,0) Click East 421
dictGet test_01037.dict_tuple (0.99,2.99) Click North 422
dictGet test_01037.dict_tuple (1,0) Click East 421
dictGet test_01037.dict_tuple (2,4) House 523
dictGet test_01037.dict_tuple (2,4.1) qqq 101
dictGet test_01037.dict_tuple (3,3) House 523
dictGet test_01037.dict_tuple (4,4) House 523
dictGet test_01037.dict_tuple (5,6) qqq 101
dictGet test_01037.dict_tuple (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_tuple (-100,-42) www 1234
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (2,4) House 523
dictGetOrDefault test_01037.dict_tuple (2,4.1) www 1234
dictGetOrDefault test_01037.dict_tuple (3,3) House 523
dictGetOrDefault test_01037.dict_tuple (4,4) House 523
dictGetOrDefault test_01037.dict_tuple (5,6) www 1234
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_tuple (-100,-42) dd 44
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (2,4) House 523
dictGetOrDefault test_01037.dict_tuple (2,4.1) gac 803
dictGetOrDefault test_01037.dict_tuple (3,3) House 523
dictGetOrDefault test_01037.dict_tuple (4,4) House 523
dictGetOrDefault test_01037.dict_tuple (5,6) cc 33
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) ee 55
dictHas test_01037.dict_array (-100,-42) 0
dictHas test_01037.dict_array (-1,0) 1
dictHas test_01037.dict_array (-0.1,0) 1
dictHas test_01037.dict_array (0,-2) 1
dictHas test_01037.dict_array (0,-1.1) 1
dictHas test_01037.dict_array (0,-1) 1
dictHas test_01037.dict_array (0,0) 1
dictHas test_01037.dict_array (0,1) 1
dictHas test_01037.dict_array (0,1.1) 1
dictHas test_01037.dict_array (0,2) 1
dictHas test_01037.dict_array (0.1,0) 1
dictHas test_01037.dict_array (0.99,2.99) 1
dictHas test_01037.dict_array (1,0) 1
dictHas test_01037.dict_array (1,1) 1
dictHas test_01037.dict_array (1,3) 1
dictHas test_01037.dict_array (2,4) 1
dictHas test_01037.dict_array (2,4.1) 0
dictHas test_01037.dict_array (3,3) 1
dictHas test_01037.dict_array (4,4) 1
dictHas test_01037.dict_array (5,1) 1
dictHas test_01037.dict_array (5,5) 1
dictHas test_01037.dict_array (5,6) 0
dictHas test_01037.dict_array (7.01,7.01) 0
dictHas test_01037.dict_tuple (-100,-42) 0
dictHas test_01037.dict_tuple (-1,0) 1
dictHas test_01037.dict_tuple (-0.1,0) 1
dictHas test_01037.dict_tuple (0,-2) 1
dictHas test_01037.dict_tuple (0,-1.1) 1
dictHas test_01037.dict_tuple (0,-1) 1
dictHas test_01037.dict_tuple (0,0) 1
dictHas test_01037.dict_tuple (0,1) 1
dictHas test_01037.dict_tuple (0,1.1) 1
dictHas test_01037.dict_tuple (0,2) 1
dictHas test_01037.dict_tuple (0.1,0) 1
dictHas test_01037.dict_tuple (0.99,2.99) 1
dictHas test_01037.dict_tuple (1,0) 1
dictHas test_01037.dict_tuple (1,1) 1
dictHas test_01037.dict_tuple (1,3) 1
dictHas test_01037.dict_tuple (2,4) 1
dictHas test_01037.dict_tuple (2,4.1) 0
dictHas test_01037.dict_tuple (3,3) 1
dictHas test_01037.dict_tuple (4,4) 1
dictHas test_01037.dict_tuple (5,1) 1
dictHas test_01037.dict_tuple (5,5) 1
dictHas test_01037.dict_tuple (5,6) 0
dictHas test_01037.dict_tuple (7.01,7.01) 0

View File

@ -0,0 +1,108 @@
SET send_logs_level = 'none';
DROP DATABASE IF EXISTS test_01037;
CREATE DATABASE test_01037 Engine = Ordinary;
DROP DICTIONARY IF EXISTS test_01037.dict_array;
DROP TABLE IF EXISTS test_01037.polygons_array;
CREATE TABLE test_01037.polygons_array (key Array(Array(Float64)), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_array VALUES ([[3, 1], [0, 1], [0, -1], [3, -1]], 'Click East', 421);
INSERT INTO test_01037.polygons_array VALUES ([[-1, 1], [1, 1], [1, 3], [-1, 3]], 'Click North', 422);
INSERT INTO test_01037.polygons_array VALUES ([[-3, 1], [-3, -1], [0, -1], [0, 1]], 'Click South', 423);
INSERT INTO test_01037.polygons_array VALUES ([[-1, -1], [1, -1], [1, -3], [-1, -3]], 'Click West', 424);
INSERT INTO test_01037.polygons_array VALUES ([[1, 1], [1, 3], [3, 5], [5, 5], [5, 1]], 'House', 523);
CREATE DICTIONARY test_01037.dict_array
(
key Array(Array(Float64)),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'polygons_array' PASSWORD '' DB 'test_01037'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(POLYGON());
DROP DICTIONARY IF EXISTS test_01037.dict_tuple;
DROP TABLE IF EXISTS test_01037.polygons_tuple;
CREATE TABLE test_01037.polygons_tuple (key Array(Tuple(Float64, Float64)), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_tuple VALUES ([(3.0, 1.0), (0.0, 1.0), (0.0, -1.0), (3.0, -1.0)], 'Click East', 421);
INSERT INTO test_01037.polygons_tuple VALUES ([(-1, 1), (1, 1), (1, 3), (-1, 3)], 'Click North', 422);
INSERT INTO test_01037.polygons_tuple VALUES ([(-3, 1), (-3, -1), (0, -1), (0, 1)], 'Click South', 423);
INSERT INTO test_01037.polygons_tuple VALUES ([(-1, -1), (1, -1), (1, -3), (-1, -3)], 'Click West', 424);
INSERT INTO test_01037.polygons_tuple VALUES ([(1, 1), (1, 3), (3, 5), (5, 5), (5, 1)], 'House', 523);
CREATE DICTIONARY test_01037.dict_tuple
(
key Array(Tuple(Float64, Float64)),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'polygons_tuple' PASSWORD '' DB 'test_01037'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(POLYGON());
DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
INSERT INTO test_01037.points VALUES (0.1, 0.0, 112, 'aax');
INSERT INTO test_01037.points VALUES (-0.1, 0.0, 113, 'aay');
INSERT INTO test_01037.points VALUES (0.0, 1.1, 114, 'aaz');
INSERT INTO test_01037.points VALUES (0.0, -1.1, 115, 'aat');
INSERT INTO test_01037.points VALUES (3.0, 3.0, 22, 'bb');
INSERT INTO test_01037.points VALUES (5.0, 6.0, 33, 'cc');
INSERT INTO test_01037.points VALUES (-100.0, -42.0, 44, 'dd');
INSERT INTO test_01037.points VALUES (7.01, 7.01, 55, 'ee')
INSERT INTO test_01037.points VALUES (0.99, 2.99, 66, 'ee');
INSERT INTO test_01037.points VALUES (1.0, 0.0, 771, 'ffa');
INSERT INTO test_01037.points VALUES (-1.0, 0.0, 772, 'ffb');
INSERT INTO test_01037.points VALUES (0.0, 2.0, 773, 'ffc');
INSERT INTO test_01037.points VALUES (0.0, -2.0, 774, 'ffd');
INSERT INTO test_01037.points VALUES (2.0, 4.0, 801, 'gaa')
INSERT INTO test_01037.points VALUES (4.0, 4.0, 802, 'gab')
INSERT INTO test_01037.points VALUES (2.0, 4.1, 803, 'gac')
select 'dictGet', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
select 'dictGet', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
INSERT INTO test_01037.points VALUES (5.0, 5.0, 0, '');
INSERT INTO test_01037.points VALUES (5.0, 1.0, 0, '');
INSERT INTO test_01037.points VALUES (1.0, 3.0, 0, '');
INSERT INTO test_01037.points VALUES (0.0, 0.0, 0, '');
INSERT INTO test_01037.points VALUES (0.0, 1.0, 0, '');
INSERT INTO test_01037.points VALUES (0.0, -1.0, 0, '');
INSERT INTO test_01037.points VALUES (1.0, 1.0, 0, '');
select 'dictHas', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
select 'dictHas', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
DROP DICTIONARY test_01037.dict_array;
DROP DICTIONARY test_01037.dict_tuple;
DROP TABLE test_01037.polygons_array;
DROP TABLE test_01037.polygons_tuple;
DROP TABLE test_01037.points;
DROP DATABASE test_01037;

View File

@ -0,0 +1,11 @@
drop table if exists t;
create table t (a Int) engine = MergeTree order by a;
-- some magic to satisfy conditions to run optimizations in MergeTreeRangeReader
insert into t select number < 20 ? 0 : 1 from numbers(50);
alter table t add column s String default 'foo';
select s from t prewhere a != 1 where rand() % 2 = 0 limit 1;
drop table t;

View File

@ -7,7 +7,22 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
RUN apt-get update \
&& DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \
p7zip-full bash curl git moreutils ncdu wget psmisc python3 python3-pip tzdata tree python3-dev g++ \
bash \
curl \
g++ \
git \
moreutils \
ncdu \
p7zip-full \
psmisc \
python3 \
python3-dev \
python3-pip \
rsync \
tree \
tzdata \
vim \
wget \
&& pip3 --no-cache-dir install clickhouse_driver \
&& apt-get purge --yes python3-dev g++ \
&& apt-get autoremove --yes \

View File

@ -45,7 +45,10 @@ function configure
sed -i 's/<tcp_port>9000/<tcp_port>9001/g' left/config/config.xml
sed -i 's/<tcp_port>9000/<tcp_port>9002/g' right/config/config.xml
cat > right/config/config.d/zz-perf-test-tweaks.xml <<EOF
mkdir right/config/users.d ||:
mkdir left/config/users.d ||:
cat > right/config/config.d/zz-perf-test-tweaks-config.xml <<EOF
<yandex>
<logger>
<console>true</console>
@ -59,7 +62,20 @@ function configure
</yandex>
EOF
cp right/config/config.d/zz-perf-test-tweaks.xml left/config/config.d/zz-perf-test-tweaks.xml
cat > right/config/users.d/zz-perf-test-tweaks-users.xml <<EOF
<yandex>
<profiles>
<default>
<query_profiler_real_time_period_ns>10000000</query_profiler_real_time_period_ns>
<query_profiler_cpu_time_period_ns>0</query_profiler_cpu_time_period_ns>
<allow_introspection_functions>1</allow_introspection_functions>
</default>
</profiles>
</yandex>
EOF
cp right/config/config.d/zz-perf-test-tweaks-config.xml left/config/config.d/zz-perf-test-tweaks-config.xml
cp right/config/users.d/zz-perf-test-tweaks-users.xml left/config/users.d/zz-perf-test-tweaks-users.xml
rm left/config/config.d/metric_log.xml ||:
rm left/config/config.d/text_log.xml ||:
@ -81,6 +97,13 @@ EOF
left/clickhouse client --port 9001 --query "create database test" ||:
left/clickhouse client --port 9001 --query "rename table datasets.hits_v1 to test.hits" ||:
while killall clickhouse ; do echo . ; sleep 1 ; done
echo all killed
# Remove logs etc, because they will be updated, and sharing them between
# servers with hardlink might cause unpredictable behavior.
rm db0/data/system/* -rf ||:
}
function restart
@ -125,11 +148,36 @@ function run_tests
rm -v test-times.tsv ||:
# Why the ugly cut:
# 1) can't make --out-format='%n' work for deleted files, it outputs things
# like "deleted 1.xml";
# 2) the output is not tab separated, but at least it's fixed width, so I
# cut by characters.
changed_files=$(rsync --dry-run --dirs --checksum --delete --itemize-changes left/performance/ right/performance/ | cut -c13-)
# FIXME remove some broken long tests
rm right/performance/{IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format}.xml ||:
test_files=$(ls right/performance/*)
# FIXME a quick crutch to bring the run time down for the flappy tests --
# run only those that have changed. Only on my prs for now.
if grep Kuzmenkov right-commit.txt
then
if [ "PR_TO_TEST" != "0" ]
then
test_files=$(cd right/performance && readlink -e $changed_files)
fi
fi
# Run only explicitly specified tests, if any
if [ -v CHPC_TEST_GLOB ]
then
test_files=$(ls right/performance/${CHPC_TEST_GLOB}.xml)
fi
# Run the tests
for test in right/performance/${CHPC_TEST_GLOB:-*}.xml
for test in $test_files
do
test_name=$(basename $test ".xml")
echo test $test_name
@ -138,8 +186,19 @@ function run_tests
{ time "$script_dir/perf.py" "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; } 2>&1 >/dev/null | grep -v ^+ >> "wall-clock-times.tsv" || continue
grep ^query "$test_name-raw.tsv" | cut -f2- > "$test_name-queries.tsv"
grep ^client-time "$test_name-raw.tsv" | cut -f2- > "$test_name-client-time.tsv"
right/clickhouse local --file "$test_name-queries.tsv" --structure 'query text, run int, version UInt32, time float' --query "$(cat $script_dir/eqmed.sql)" > "$test_name-report.tsv"
# this may be slow, run it in background
right/clickhouse local --file "$test_name-queries.tsv" --structure 'query text, run int, version UInt32, time float' --query "$(cat $script_dir/eqmed.sql)" > "$test_name-report.tsv" &
done
wait
# Collect the profiles
left/clickhouse client --port 9001 --query "select * from system.trace_log format TSVWithNamesAndTypes" > left-trace-log.tsv ||: &
left/clickhouse client --port 9001 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > left-addresses.tsv ||: &
right/clickhouse client --port 9002 --query "select * from system.trace_log format TSVWithNamesAndTypes" > right-trace-log.tsv ||: &
right/clickhouse client --port 9002 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > right-addresses.tsv ||: &
wait
}
# Analyze results
@ -156,7 +215,7 @@ create table queries engine Memory as select
-- remove them altogether because we want to be able to detect regressions,
-- but the right way to do this is not yet clear.
not short and abs(diff) < 0.05 and rd[3] > 0.05 as unstable,
not short and abs(diff) > 0.05 and abs(diff) > rd[3] as changed,
not short and abs(diff) > 0.10 and abs(diff) > rd[3] as changed,
*
from file('*-report.tsv', TSV, 'left float, right float, diff float, rd Array(float), query text');
@ -201,7 +260,7 @@ create table test_times_tsv engine File(TSV, 'test-times.tsv') as
floor(real / queries, 3) avg_real_per_query,
floor(query_min, 3)
from test_time join wall_clock using test
order by query_max / query_min desc;
order by avg_real_per_query desc;
create table all_queries_tsv engine File(TSV, 'all-queries.tsv') as
select left, right, diff, rd, test, query

View File

@ -8,7 +8,7 @@ select
query
from
(
select query, quantiles(0.05, 0.5, 0.95)(abs(time_by_label[1] - time_by_label[2])) rd_quantiles -- quantiles of randomization distribution
select query, quantiles(0.05, 0.5, 0.95, 0.99)(abs(time_by_label[1] - time_by_label[2])) rd_quantiles -- quantiles of randomization distribution
from
(
select query, virtual_run, groupArrayInsertAt(median_time, random_label) time_by_label -- make array 'random label' -> 'median time'

View File

@ -10,6 +10,15 @@ import pprint
import time
import traceback
stage_start_seconds = time.perf_counter()
def report_stage_end(stage_name):
global stage_start_seconds
print('{}\t{}'.format(stage_name, time.perf_counter() - stage_start_seconds))
stage_start_seconds = time.perf_counter()
report_stage_end('start')
parser = argparse.ArgumentParser(description='Run performance test.')
# Explicitly decode files as UTF-8 because sometimes we have Russian characters in queries, and LANG=C is set.
parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file')
@ -35,6 +44,8 @@ if infinite_sign is not None:
servers = [{'host': host, 'port': port} for (host, port) in zip(args.host, args.port)]
connections = [clickhouse_driver.Client(**server) for server in servers]
report_stage_end('connect')
# Check tables that should exist
tables = [e.text for e in root.findall('preconditions/table_exists')]
for t in tables:
@ -47,6 +58,8 @@ for c in connections:
for s in settings:
c.execute("set {} = '{}'".format(s.tag, s.text))
report_stage_end('preconditions')
# Process substitutions
subst_elems = root.findall('substitutions/substitution')
@ -61,6 +74,8 @@ parameter_combinations = [dict(zip(parameter_keys, parameter_combination)) for p
def substitute_parameters(query_templates, parameter_combinations):
return list(set([template.format(**parameters) for template, parameters in itertools.product(query_templates, parameter_combinations)]))
report_stage_end('substitute')
# Run drop queries, ignoring errors
drop_query_templates = [q.text for q in root.findall('drop_query')]
drop_queries = substitute_parameters(drop_query_templates, parameter_combinations)
@ -86,6 +101,8 @@ for c in connections:
for q in fill_queries:
c.execute(q)
report_stage_end('fill')
# Run test queries
def tsv_escape(s):
return s.replace('\\', '\\\\').replace('\t', '\\t').replace('\n', '\\n').replace('\r','')
@ -93,6 +110,8 @@ def tsv_escape(s):
test_query_templates = [q.text for q in root.findall('query')]
test_queries = substitute_parameters(test_query_templates, parameter_combinations)
report_stage_end('substitute2')
for q in test_queries:
# Prewarm: run once on both servers. Helps to bring the data into memory,
# precompile the queries, etc.
@ -115,9 +134,13 @@ for q in test_queries:
client_seconds = time.perf_counter() - start_seconds
print('client-time\t{}\t{}\t{}'.format(tsv_escape(q), client_seconds, server_seconds))
report_stage_end('benchmark')
# Run drop queries
drop_query_templates = [q.text for q in root.findall('drop_query')]
drop_queries = substitute_parameters(drop_query_templates, parameter_combinations)
for c in connections:
for q in drop_queries:
c.execute(q)
report_stage_end('drop')

View File

@ -147,8 +147,8 @@ params['test_part'] = (
anchor = nextTableAnchor(),
caption = 'Tests times',
header = table_header(['Test', 'Wall clock time, s', 'Total client time, s',
'Number of queries',
'Number of short queries',
'Total queries',
'Ignored short queries',
'Longest query<br>(sum for all runs), s',
'Avg wall clock time<br>(sum for all runs), s',
'Shortest query<br>(sum for all runs), s']),

View File

@ -114,16 +114,16 @@ Features:
- SQL data modeling support for relational mapping of data.
### Looker
[Looker](https://looker.com) is a data platform and business intelligence tool with support for 50+ database dialects including ClickHouse. Looker is available as a SaaS platform and self-hosted. Users
can use Looker via the browser to explore data, build visualizations and dashboards, schedule reports, and share their
insights with colleagues. Looker provides a rich set of tools to embed these features in other applications, and an API
[Looker](https://looker.com) is a data platform and business intelligence tool with support for 50+ database dialects including ClickHouse. Looker is available as a SaaS platform and self-hosted. Users can use Looker via the browser to explore data, build visualizations and dashboards, schedule reports, and share their insights with colleagues. Looker provides a rich set of tools to embed these features in other applications, and an API
to integrate data with other applications.
Features:
- Designed around ease of use and self-service for end users.
- Easy and agile development using LookML, a language which supports currated
[Data Modeling](https://looker.com/platform/data-modeling) to support report writers and end users.
- Powerful workflow integration via Looker's [Data Actions](https://looker.com/platform/actions).
[How to configure ClickHouse in Looker.](https://docs.looker.com/setup-and-management/database-config/clickhouse)
[Original article](https://clickhouse.tech/docs/en/interfaces/third-party/gui/) <!--hide-->

View File

@ -0,0 +1,11 @@
# ClickHouse Adopters
!!! note "Disclaimer"
The following list of companies using ClickHouse and their success stories is assembled from public sources, thus might differ from current reality. We'd really appreciate if you share the story of adopting ClickHouse in your company and [add it to the list](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md), but please make sure you won't have any NDA issues by doing so. Providing updates with publications by other companies is also useful.
| Company | Industry | Usecase | Cluster Size | Compressed/Uncompressed Data Size<abbr title="without replication"><sup>*</sup></abbr> | Reference |
| --- | --- | --- | --- | --- | --- |
| [Badoo](https://badoo.com) | Dating | Timeseries | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/ml.pdf) |
| [Uber](https://www.uber.com) | Taxi | Logging | — | — | [Slides in English, February 2020](https://presentations.clickhouse.tech/meetup40/ml.pdf) |
| [Yandex DataLens](https://cloud.yandex.ru/services/datalens) | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/datalens.pdf) |
| [Yandex Metrica](https://metrica.yandex.com) | Web analytics | Main product | 360 servers in one cluster, 1862 servers in one department | 66.41 PiB / 5.68 PiB | [Slides in English, February 2020](https://presentations.clickhouse.tech/meetup40/introduction/#13) |

View File

@ -564,29 +564,29 @@ ClickHouse doesn't directly support syntax with commas, so we don't recommend us
Tables for `ASOF JOIN` must have an ordered sequence column. This column cannot be alone in a table, and should be one of the data types: `UInt32`, `UInt64`, `Float32`, `Float64`, `Date`, and `DateTime`.
You can use the following types of syntax:
Syntax `ASOF JOIN ... ON`:
- `ASOF JOIN ... ON`
```sql
SELECT expressions_list
FROM table_1
ASOF LEFT JOIN table_2
ON equi_cond AND closest_match_cond
```
```sql
SELECT expressions_list
FROM table_1
ASOF LEFT JOIN table_2
ON equi_cond AND closest_match_cond
```
You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`.
You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t`. Only `table_2.some_col <= table_1.some_col` and `table_1.some_col >= table2.some_col` condition types are available. You can't apply other conditions like `>` or `!=`.
Conditions supported for the closest match: `>`, `>=`, `<`, `<=`.
- `ASOF JOIN ... USING`
Syntax `ASOF JOIN ... USING`:
```sql
SELECT expressions_list
FROM table_1
ASOF JOIN table_2
USING (equi_column1, ... equi_columnN, asof_column)
```
```sql
SELECT expressions_list
FROM table_1
ASOF JOIN table_2
USING (equi_column1, ... equi_columnN, asof_column)
```
`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table2.asof_column` condition. The `asof_column` column must be the last in the `USING` clause.
`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column always the last one in the `USING` clause.
For example, consider the following tables:
@ -602,7 +602,7 @@ event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42
... ...
```
`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest (equal to or less) to the timestamp of the event from `table_1`. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can't be joined.
`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can't be joined.
!!! note "Note"

View File

@ -38,7 +38,7 @@ $ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'"
Displays a list of tables.
```sql
SHOW [TEMPORARY] TABLES [FROM <db>] [LIKE '<pattern>'] [LIMIT <N>] [INTO OUTFILE <filename>] [FORMAT <format>]
SHOW [TEMPORARY] TABLES [{FROM | IN} <db>] [LIKE '<pattern>' | WHERE expr] [LIMIT <N>] [INTO OUTFILE <filename>] [FORMAT <format>]
```
If the `FROM` clause is not specified, the query returns the list of tables from the current database.

View File

@ -0,0 +1 @@
../../en/development/build_cross_arm.md

View File

@ -0,0 +1 @@
../../en/development/build_cross_osx.md

View File

@ -0,0 +1 @@
../../en/guides/apply_catboost_model.md

1
docs/fa/guides/index.md Symbolic link
View File

@ -0,0 +1 @@
../../en/guides/index.md

1
docs/fa/interfaces/mysql.md Symbolic link
View File

@ -0,0 +1 @@
../../en/interfaces/mysql.md

View File

@ -0,0 +1 @@
../../en/introduction/adopters.md

1
docs/ja/interfaces/mysql.md Symbolic link
View File

@ -0,0 +1 @@
../../en/interfaces/mysql.md

View File

@ -0,0 +1 @@
../../en/introduction/adopters.md

View File

@ -1 +0,0 @@
../../en/introduction/distinctive_features.md

View File

@ -0,0 +1,61 @@
# ClickHouseの特徴
## 真の列指向DBMS
真の列指向DBMSでは、値に関する余分なデータは保存されません。 これは、固定長の値がサポートされ、「データ長」が格納されないことを意味します。 例として、10億個のUInt8型の値は、圧縮されていない状態で約1 GBを消費しますが、そうしないとCPU負荷に大きく影響します。 これは、データの展開速度(CPU使用率)が、主に非圧縮データのサイズに依存するため、データを(冗長なデータが含まれないように)コンパクトに格納することが非常に重要であるためです。
値を列ごとに保存できるシステムは他にもありますが、それらは「分析クエリの効果的な処理」以外のシナリオに最適化されているため、ClickHouseは注目に値します。 例えば、HBase、BigTable、Cassandra、およびHyperTableなどです。これらのシステムでは、毎秒数十万行のスループットが得られますが、毎秒数億行ではありません。
また、ClickHouseは単一のデータベースではなく、データベース管理システムであることに注意してください。 ClickHouseを使用すると、サーバーを再構成および再起動することなく、実行時にテーブルとデータベースを作成し、データを読み込み、クエリを実行できます。
## データ圧縮
一部の列指向DBMSInfiniDB CEおよびMonetDBは、データ圧縮を使用しません。ただし、優れたパフォーマンスを実現するには、データ圧縮が重要な役割を果たします。
## データのディスクストレージ
データを主キーで物理的に並べ替えておくと、特定の値または値範囲のデータを数十ミリ秒未満の低遅延で抽出できます。一部の列指向DBMSSAP HANAやGoogle PowerDrillなどは、RAMでのみ機能します。このアプローチでは、リアルタイム分析を行う上で、実際に必要なハードウェアよりも規模の大きな予算の割り当てが推奨されます。 これに対しClickHouseは、通常のハードドライブで動作するように設計されています。 つまり、データストレージのGBあたりのコストは低くなり、さらに利用可能なSSDや追加のRAMがあれば、それらもフルに使用されます。
## マルチコアでの並列処理
大規模なクエリは、現在のサーバーで使用可能なすべての必要なリソースを使用して、自然な方法で並列化されます。
## 複数のサーバーでの分散処理
前述の列指向DBMSのほとんどは、分散クエリ処理をサポートしていません。 ClickHouseでは、データを異なるシャードに置くことができます。各シャードは、フォールトトレランスに使用されるレプリカのグループにすることができます。クエリは、すべてのシャードで並行して処理されます。これはユーザーにとって透過的です。
## SQLのサポート
ClickHouseは、多くの場合、標準SQLと同一のSQLに基づく宣言的なクエリ言語をサポートしています。サポートされるクエリには、GROUP BY、ORDER BYや、FROM、IN、およびJOIN句中のサブクエリ、およびスカラーサブクエリが含まれます。従属サブクエリとウィンドウ関数はサポートされていません。
## ベクトルエンジン
データは列ごとに格納されるだけでなく、ベクトル列の一部によって処理されます。これにより、高いCPU効率を実現できます。
## リアルタイムのデータ更新
ClickHouseは、主キーを持つテーブルをサポートします。 主キーの範囲でクエリをすばやく実行するために、データはマージツリーを使用して随時並べ替えられます。これにより、データを継続的にテーブルに追加できます。新しいデータが取り込まれたときにロックは取得されません。
## インデックス
データを主キーで物理的に並べ替えることにより、特定の値または値範囲のデータを数十ミリ秒未満の低遅延で抽出できます。
## オンラインクエリとの親和性
低レイテンシであるとは、UIページの読み込み中に、事前計算を伴わないでクエリを遅延なく処理できることを意味します。これはオンラインであるとも言えます。
## 近似計算のサポート
ClickHouseには、精度を犠牲にしてパフォーマンスを得るための様々な方法が用意されています。
1. ユニーク値、中央値、および分位数を近似計算のための集計関数。
2. データの一部(サンプル)に基づいてクエリを実行し、近似結果を取得します。この場合、ディスクから取得されるデータは近似に比例して少なくなります。
3. すべてのキーではなく、限られた数のランダムキーに対して集計を実行します。データ内のキー分布が特定の条件下であれば、使用するリソースが少なくなり、かなり正確な結果が得られます。
## データのレプリケーションとデータ整合性のサポート
非同期マルチマスターレプリケーションを使用します。使用可能なレプリカに書き込まれた後、データはバックグラウンドで残りのすべてのレプリカに配布されます。システムは、異なるレプリカで同一のデータを維持します。ほとんどの障害後の回復は、自動的に、複雑な場合には半自動的に実行されます。
詳細については、[データ複製](../operations/table_engines/replication.md) セクションを参照してください。
[Original article](https://clickhouse.yandex/docs/en/introduction/distinctive_features/) <!--hide-->

View File

@ -0,0 +1 @@
../../en/database_engines/lazy.md

View File

@ -0,0 +1 @@
../../en/development/build_cross_arm.md

View File

@ -117,3 +117,19 @@
- Моделирование данных с помощью SQL для их реляционного отображения.
[Оригинальная статья](https://clickhouse.tech/docs/ru/interfaces/third-party/gui/) <!--hide-->
### Looker
[Looker](https://looker.com) — платформа для обработки данных и бизнес-аналитики. Поддерживает более 50 диалектов баз данных, включая ClickHouse. Looker можно установить самостоятельно или воспользоваться готовой платформой SaaS.
Просмотр данных, построение отображений и дашбордов, планирование отчётов и обмен данными с коллегами доступны с помощью браузера. Также, Looker предоставляет ряд инструментов, позволяющих встраивать сервис в другие приложения и API для обмена данными.
Основные возможности:
- Язык LookML, поддерживающий [моделирование данных](https://looker.com/platform/data-modeling).
- Интеграция с различными системами с помощью [Data Actions](https://looker.com/platform/actions).
- Инструменты для встраивания сервиса в приложения.
- API.
[Как сконфигурировать ClickHouse в Looker.](https://docs.looker.com/setup-and-management/database-config/clickhouse)
[Original article](https://clickhouse.tech/docs/ru/interfaces/third-party/gui/) <!--hide-->

View File

@ -0,0 +1 @@
../../en/introduction/adopters.md

View File

@ -0,0 +1 @@
../../../en/operations/utils/clickhouse-benchmark.md

View File

@ -572,29 +572,29 @@ ClickHouse не поддерживает синтаксис с запятыми
Таблицы для `ASOF JOIN` должны иметь столбец с отсортированной последовательностью. Этот столбец не может быть единственным в таблице и должен быть одного из типов: `UInt32`, `UInt64`, `Float32`, `Float64`, `Date` и `DateTime`.
Можно использовать следующие типы синтаксиса:
Синтаксис `ASOF JOIN ... ON`:
- `ASOF JOIN ... ON`
```sql
SELECT expressions_list
FROM table_1
ASOF LEFT JOIN table_2
ON equi_cond AND closest_match_cond
```
```sql
SELECT expressions_list
FROM table_1
ASOF LEFT JOIN table_2
ON equi_cond AND closest_match_cond
```
Можно использовать произвольное количество условий равенства и одно условие на ближайшее совпадение. Например, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`.
Можно использовать произвольное количество условий равенства и одно условие на ближайшее совпадение. Например, `SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t`. Можно использовать только условия `table_2.some_col <= table_1.some_col` и `table_1.some_col >= table2.some_col`. Условия типа `>` или `!=` не поддерживаются.
Условия, поддержанные для проверки на ближайшее совпадение: `>`, `>=`, `<`, `<=`.
- `ASOF JOIN ... USING`
Синтаксис `ASOF JOIN ... USING`:
```sql
SELECT expressions_list
FROM table_1
ASOF JOIN table_2
USING (equi_column1, ... equi_columnN, asof_column)
```
```sql
SELECT expressions_list
FROM table_1
ASOF JOIN table_2
USING (equi_column1, ... equi_columnN, asof_column)
```
Для слияния по равенству `ASOF JOIN` использует `equi_columnX`, а для слияния по ближайшему совпадению использует `asof_column` с условием `table_1.asof_column >= table2.asof_column`. Столбец `asof_column` должен быть последним в секции `USING`.
Для слияния по равенству `ASOF JOIN` использует `equi_columnX`, а для слияния по ближайшему совпадению использует `asof_column` с условием `table_1.asof_column >= table_2.asof_column`. Столбец `asof_column` должен быть последним в секции `USING`.
Например, рассмотрим следующие таблицы:
@ -609,7 +609,7 @@ event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42
... ...
```
`ASOF JOIN` принимает метку времени пользовательского события из `table_1` и находит такое событие в `table_2` метка времени которого наиболее близка (равна или меньше) к метке времени события из `table_1`. При этом столбец `user_id` используется для объединения по равенству, а столбец `ev_time` для объединения по ближайшему совпадению. В нашем примере `event_1_1` может быть объединено с `event_2_1`, `event_1_2` может быть объединено с `event_2_3`, а `event_2_2` не объединяется.
`ASOF JOIN` принимает метку времени пользовательского события из `table_1` и находит такое событие в `table_2` метка времени которого наиболее близка к метке времени события из `table_1` в соответствии с условием на ближайшее совпадение. При этом столбец `user_id` используется для объединения по равенству, а столбец `ev_time` для объединения по ближайшему совпадению. В нашем примере `event_1_1` может быть объединено с `event_2_1`, `event_1_2` может быть объединено с `event_2_3`, а `event_2_2` не объединяется.
!!! note "Примечание"
`ASOF JOIN` не поддержан для движка таблиц [Join](../operations/table_engines/join.md).

View File

@ -6,6 +6,7 @@ nav:
- 'ClickHouse Features that Can Be Considered Disadvantages': 'introduction/features_considered_disadvantages.md'
- 'Performance': 'introduction/performance.md'
- 'History': 'introduction/history.md'
- 'Adopters': 'introduction/adopters.md'
- 'Getting Started':
- 'hidden': 'getting_started/index.md'
@ -219,7 +220,7 @@ nav:
- 'clickhouse-local': 'operations/utils/clickhouse-local.md'
- 'clickhouse-benchmark': 'operations/utils/clickhouse-benchmark.md'
- 'Usage Recommendations': 'operations/tips.md'
- 'Development':
- 'hidden': 'development/index.md'
- 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md'

View File

@ -6,6 +6,7 @@ nav:
- ' ویژگی های از ClickHouse که می تواند معایبی باشد': 'introduction/features_considered_disadvantages.md'
- 'ﯽﯾﺍﺭﺎﮐ': 'introduction/performance.md'
- 'ﺦﯾﺭﺎﺗ': 'introduction/history.md'
- 'Adopters': 'introduction/adopters.md'
- 'Getting started':
- 'hidden': 'getting_started/index.md'

View File

@ -6,6 +6,7 @@ nav:
- 'ClickHouse Features that Can Be Considered Disadvantages': 'introduction/features_considered_disadvantages.md'
- 'Performance': 'introduction/performance.md'
- 'History': 'introduction/history.md'
- 'Adopters': 'introduction/adopters.md'
- 'Getting Started':
- 'hidden': 'getting_started/index.md'

View File

@ -7,6 +7,7 @@ nav:
- 'Производительность': 'introduction/performance.md'
- 'История': 'introduction/history.md'
- 'Информационная поддержка': 'introduction/info.md'
- 'Пользователи': 'introduction/adopters.md'
- 'Начало работы':
- 'hidden': 'getting_started/index.md'

View File

@ -6,6 +6,7 @@ nav:
- 'ClickHouse功能可被视为缺点': 'introduction/features_considered_disadvantages.md'
- '性能': 'introduction/performance.md'
- '历史': 'introduction/history.md'
- '使用者': 'introduction/adopters.md'
- '入门指南':
- 'hidden': 'getting_started/index.md'

View File

@ -294,6 +294,7 @@ if __name__ == '__main__':
arg_parser.add_argument('--skip-single-page', action='store_true')
arg_parser.add_argument('--skip-pdf', action='store_true')
arg_parser.add_argument('--skip-website', action='store_true')
arg_parser.add_argument('--minify', action='store_true')
arg_parser.add_argument('--save-raw-single-page', type=str)
arg_parser.add_argument('--verbose', action='store_true')

View File

@ -13,9 +13,10 @@ function do_make_links()
# replacing "/./" with /
dst_file="../${lang}${src_file}"
dst_file="${dst_file/\/\.\//\/}"
dst_file="${dst_file/${lang}\./${lang}}"
mkdir -p $(dirname "${dst_file}")
ln -sr "${src_file}" "${dst_file}"
ln -sr "${src_file}" "${dst_file}" 2>/dev/null
done
}

View File

@ -1,78 +1,9 @@
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot);
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/CYblzLEXzCqQIvrYs7QKQe2omRk.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/pUcnOdRwl83MvPPzrNomhyletnA.woff) format('woff'),
url(https://yastatic.net/adv-www/_/vNFEmXOcGYKJ4AAidUprHWoXrLU.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/0w7OcWZM_QLP8x-LQUXFOgXO6dE.svg#YandexSansTextWeb-Bold) format('svg');
font-weight: 700;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot);
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/z3MYElcut0R2MF_Iw1RDNrstgYs.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/1jvKJ_-hCXl3s7gmFl-y_-UHTaI.woff) format('woff'),
url(https://yastatic.net/adv-www/_/9nzjfpCR2QHvK1EzHpDEIoVFGuY.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/gwyBTpxSwkFCF1looxqs6JokKls.svg#YandexSansTextWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot);
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/lGQcYklLVV0hyvz1HFmFsUTj8_0.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/f0AAJ9GJ4iiwEmhG-7PWMHk6vUY.woff) format('woff'),
url(https://yastatic.net/adv-www/_/4UDe4nlVvgEJ-VmLWNVq3SxCsA.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/EKLr1STNokPqxLAQa_RyN82pL98.svg#YandexSansTextWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot);
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/sUYVCPUAQE7ExrvMS7FoISoO83s.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/v2Sve_obH3rKm6rKrtSQpf-eB7U.woff) format('woff'),
url(https://yastatic.net/adv-www/_/PzD8hWLMunow5i3RfJ6WQJAL7aI.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/lF_KG5g4tpQNlYIgA0e77fBSZ5s.svg#YandexSansDisplayWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot);
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/LGiRvlfqQHlWR9YKLhsw5e7KGNA.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/40vXwNl4eYYMgteIVgLP49dwmfc.woff) format('woff'),
url(https://yastatic.net/adv-www/_/X6zG5x_wO8-AtwJ-vDLJcKC5228.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/ZKhaR0m08c8CRRL77GtFKoHcLYA.svg#YandexSansDisplayWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
body {
font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif;
}
body.md-lang-zh {
font: 400 14pt/200% 'Yandex Sans Text Web', -apple-system, BlinkMacSystemFont, "Segoe UI", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol";
font: 300 14pt/200% -apple-system, BlinkMacSystemFont, Segoe UI, Helvetica, Arial, sans-serif, Apple Color Emoji,Segoe UI Emoji;
}
a:link, a:visited {
color: #08f;
color: #f14600;
text-decoration: none;
}
@ -84,26 +15,23 @@ a:link, a:visited {
color: #888 !important;
}
.md-nav__link:hover, .md-nav__link:active {
color: #08f !important;
color: #f14600 !important;
text-decoration: none;
}
a:hover, a:active {
color: #f00;
text-decoration: underline;
}
.md-typeset pre {
font: 13px/18px monospace, "Courier New";
font: 13px/18px SFMono-Regular, Consolas, Liberation Mono, Menlo, monospace;
display: block;
padding: 1rem 3rem 1rem 1rem;
overflow: scroll;
}
h1, h2, h3, .md-logo {
font-family: 'Yandex Sans Display Web', Arial, sans-serif;
color: #000 !important;
}
@ -117,12 +45,11 @@ h1, h2, h3, .md-logo {
.md-header-nav__title {
font-size: 3rem;
font-family: 'Yandex Sans Display Web', Arial, sans-serif;
}
.md-content__icon:hover {
text-decoration: none !important;
color: #08f !important;
color: #f14600 !important;
}
.md-search-result__link {
@ -229,3 +156,7 @@ h1, h2, h3, .md-logo {
overflow-x: hidden;
overflow-y: auto;
}
.md-typeset a:active, .md-typeset a:hover {
color: #f14600 !important;
}

View File

@ -4,7 +4,6 @@ set -ex
BASE_DIR=$(dirname $(readlink -f $0))
BUILD_DIR="${BASE_DIR}/../build"
PUBLISH_DIR="${BASE_DIR}/../publish"
IMAGE="clickhouse/website"
GIT_TEST_URI="git@github.com:ClickHouse/clickhouse-test.github.io.git"
GIT_PROD_URI="git@github.com:ClickHouse/clickhouse.github.io.git"
@ -14,8 +13,6 @@ then
else
TAG="$1"
fi
FULL_NAME="${IMAGE}:${TAG}"
REMOTE_NAME="registry.yandex.net/${FULL_NAME}"
DOCKER_HASH="$2"
if [[ -z "$1" ]]
then
@ -37,10 +34,7 @@ then
git commit -a -m "add new release at $(date)"
git push origin master
cd "${BUILD_DIR}"
docker build -t "${FULL_NAME}" "${BUILD_DIR}"
docker tag "${FULL_NAME}" "${REMOTE_NAME}"
DOCKER_HASH=$(docker push "${REMOTE_NAME}" | tail -1 | awk '{print $3;}')
docker rmi "${FULL_NAME}"
DOCKER_HASH=$(head -c 16 < /dev/urandom | xxd -p)
else
rm -rf "${BUILD_DIR}" || true
rm -rf "${PUBLISH_DIR}" || true
@ -67,8 +61,4 @@ then
else
QLOUD_ENV="${QLOUD_PROJECT}.prod"
fi
QLOUD_COMPONENT="${QLOUD_ENV}.nginx"
QLOUD_VERSION=$(curl -v -H "Authorization: OAuth ${QLOUD_TOKEN}" "${QLOUD_ENDPOINT}/environment/status/${QLOUD_ENV}" | python -c "import json; import sys; print json.loads(sys.stdin.read()).get('version')")
curl -v -H "Authorization: OAuth ${QLOUD_TOKEN}" -H "Content-Type: application/json" --data "{\"repository\": \"${REMOTE_NAME}\", \"hash\": \"${DOCKER_HASH}\"}" "${QLOUD_ENDPOINT}/component/${QLOUD_COMPONENT}/${QLOUD_VERSION}/deploy" > /dev/null
echo ">>> Successfully deployed ${TAG} ${DOCKER_HASH} to ${QLOUD_ENV} <<<"

View File

@ -21,25 +21,27 @@ def build_website(args):
)
)
def minify_website(args):
for root, _, filenames in os.walk(args.output_dir):
for filename in filenames:
path = os.path.join(root, filename)
if not (
filename.endswith('.html') or
filename.endswith('.css') or
filename.endswith('.js')
):
continue
logging.info('Minifying %s', path)
with open(path, 'rb') as f:
content = f.read().decode('utf-8')
if filename.endswith('.html'):
content = htmlmin.minify(content, remove_empty_space=False)
elif filename.endswith('.css'):
content = cssmin.cssmin(content)
elif filename.endswith('.js'):
content = jsmin.jsmin(content)
with open(path, 'wb') as f:
f.write(content.encode('utf-8'))
def minify_website(args):
if args.minify:
for root, _, filenames in os.walk(args.output_dir):
for filename in filenames:
path = os.path.join(root, filename)
if not (
filename.endswith('.html') or
filename.endswith('.css') or
filename.endswith('.js')
):
continue
logging.info('Minifying %s', path)
with open(path, 'rb') as f:
content = f.read().decode('utf-8')
if filename.endswith('.html'):
content = htmlmin.minify(content, remove_empty_space=False)
elif filename.endswith('.css'):
content = cssmin.cssmin(content)
elif filename.endswith('.js'):
content = jsmin.jsmin(content)
with open(path, 'wb') as f:
f.write(content.encode('utf-8'))

View File

@ -0,0 +1 @@
../../en/database_engines/lazy.md

View File

@ -0,0 +1 @@
../../en/development/build_cross_arm.md

View File

@ -0,0 +1 @@
../../en/guides/apply_catboost_model.md

1
docs/zh/guides/index.md Symbolic link
View File

@ -0,0 +1 @@
../../en/guides/index.md

1
docs/zh/interfaces/mysql.md Symbolic link
View File

@ -0,0 +1 @@
../../en/interfaces/mysql.md

View File

@ -0,0 +1 @@
../../en/introduction/adopters.md

View File

@ -0,0 +1,43 @@
#pragma once
#include <memory>
namespace ext {
template <class T>
class Singleton
{
public:
Singleton()
{
if (!instance)
instance.reset(new T);
}
template <typename ... Args>
Singleton(const Args & ... args)
{
instance.reset(new T(args...));
/// TODO: throw exception on double-creation.
}
T * operator->()
{
return instance.get();
}
static bool isInitialized()
{
return !!instance;
}
static void reset()
{
instance.reset();
}
private:
inline static std::unique_ptr<T> instance{};
};
}

View File

@ -1 +1,3 @@
# -*- coding: utf-8 -*-
# REMOVE ME

View File

@ -2154,6 +2154,8 @@ try { var yaCounter18343495 = new Ya.Metrika({id:18343495,
<p>Most results are for single server setup. The server is: two socket Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz; 128 GiB RAM; md RAID-5 on 8 6TB SATA HDD; ext4.</p>
<p>See as well <a href="benchmark_hardware.html">hardware benchmark results page</a>.</p>
<p>Some additional results (marked as x2, x3, x6) are for clustered setup for comparison. These results are contributed from independent teams and hardware specification may differ.</p>
<p>Disclaimer: some results are significantly outdated. Results for MonetDB was obtained at 2013 for version v11.15.11 (Feb2013-SP3). Results for MemSQL was obtained at 2015 for version 3.2. Results for Vertica are from 2015 for version 7.1.1.</p>

View File

@ -2019,6 +2019,57 @@ var results =
[0.006, 0.005, 0.005]
]
},
{
"system": "AWS a1.4xlarge (Graviton) 16 vCPU, 2.3 GHz, 32 GiB RAM, EBS",
"time": "2020-02-13 00:00:00",
"result":
[
[0.012, 0.003, 0.003],
[0.073, 0.031, 0.031],
[0.098, 0.053, 0.053],
[0.209, 0.139, 0.141],
[0.251, 0.200, 0.202],
[0.662, 0.439, 0.436],
[0.062, 0.041, 0.041],
[0.040, 0.033, 0.032],
[3.379, 0.720, 0.722],
[0.934, 0.847, 0.845],
[0.436, 0.379, 0.377],
[0.500, 0.417, 0.430],
[1.536, 1.381, 1.373],
[1.956, 1.832, 1.855],
[1.527, 1.458, 1.466],
[1.613, 1.576, 1.581],
[3.644, 3.490, 3.530],
[2.143, 1.982, 1.965],
[7.808, 7.617, 7.764],
[0.390, 0.179, 0.168],
[8.797, 2.308, 2.257],
[10.138, 2.533, 2.517],
[19.626, 5.738, 5.707],
[20.183, 2.195, 2.156],
[1.841, 0.577, 0.578],
[0.535, 0.479, 0.476],
[1.830, 0.578, 0.577],
[8.786, 2.521, 2.524],
[7.364, 2.941, 2.926],
[3.373, 3.186, 3.203],
[1.641, 1.213, 1.209],
[4.890, 1.964, 1.913],
[10.442, 10.410, 10.427],
[11.183, 7.431, 7.402],
[11.175, 7.460, 7.487],
[2.317, 2.232, 2.221],
[0.473, 0.406, 0.418],
[0.201, 0.187, 0.183],
[0.193, 0.144, 0.160],
[0.901, 0.811, 0.836],
[0.090, 0.046, 0.041],
[0.053, 0.032, 0.033],
[0.015, 0.012, 0.012]
]
},
];
</script>

View File

@ -0,0 +1 @@
<svg xmlns="http://www.w3.org/2000/svg" viewBox="0 0 66.12 10.29"><defs><style>.a{fill:#231f20;}</style></defs><title>ClickHouse</title><path class="a" d="M58.54,74.8a5,5,0,0,1-1.88-.34,4.26,4.26,0,0,1-1.48-1,4.46,4.46,0,0,1-1-1.55,6.32,6.32,0,0,1,0-4.21,4.39,4.39,0,0,1,2.58-2.57,5.36,5.36,0,0,1,1.93-.34,5.84,5.84,0,0,1,1.56.18,6.38,6.38,0,0,1,1.2.46l-0.36,1.13A5.17,5.17,0,0,0,60,66a4.23,4.23,0,0,0-1.43-.22,3.07,3.07,0,0,0-2.35,1.06,3.81,3.81,0,0,0-.72,1.27,5.17,5.17,0,0,0-.27,1.73,5.35,5.35,0,0,0,.25,1.69,3.61,3.61,0,0,0,.69,1.25,2.88,2.88,0,0,0,1.06.77,3.49,3.49,0,0,0,1.38.27,4,4,0,0,0,1.27-.2A5.71,5.71,0,0,0,61,73.09l0.35,0.85a3.86,3.86,0,0,1-1.12.59A5,5,0,0,1,58.54,74.8Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M64.07,64.51V74.66H62.88V64.51h1.19Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M66.64,66.55a0.71,0.71,0,0,1-.76-0.74,0.78,0.78,0,0,1,.2-0.55,0.79,0.79,0,0,1,1.1,0,0.77,0.77,0,0,1,.21.55,0.72,0.72,0,0,1-.21.53A0.74,0.74,0,0,1,66.64,66.55Zm0.6,1.06v7H66.06v-7h1.19Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M72.29,74.8a3.78,3.78,0,0,1-1.37-.25,3.08,3.08,0,0,1-1.11-.72,3.45,3.45,0,0,1-.74-1.15,4.48,4.48,0,0,1,0-3.08,3.35,3.35,0,0,1,.76-1.15A3.19,3.19,0,0,1,71,67.72a4,4,0,0,1,1.41-.25,4.68,4.68,0,0,1,1.13.13,2.68,2.68,0,0,1,.84.35L74,69a4,4,0,0,0-.79-0.47,2.32,2.32,0,0,0-.92-0.16,2.28,2.28,0,0,0-.88.17,2,2,0,0,0-.73.52,2.52,2.52,0,0,0-.49.85A3.5,3.5,0,0,0,70,71.11a2.8,2.8,0,0,0,.66,2,2.4,2.4,0,0,0,1.79.67,3,3,0,0,0,1.62-.52l0.27,0.88a2.89,2.89,0,0,1-.89.45A3.88,3.88,0,0,1,72.29,74.8Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M77,64.51V74.66H75.79V64.51H77ZM80.1,74.66L77.16,70.9l2.9-3.28H81.3l-2.87,3.14,3.07,3.9H80.1Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M89.17,70.34h-5v4.31H82.88V64.93h1.29v4.44h5V64.93h1.29v9.73H89.17V70.34Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M95.55,74.8a3.49,3.49,0,0,1-1.3-.25,3.11,3.11,0,0,1-1.08-.71,3.44,3.44,0,0,1-.73-1.15,4.6,4.6,0,0,1,0-3.12,3.36,3.36,0,0,1,.73-1.14,3.16,3.16,0,0,1,1.08-.71,3.49,3.49,0,0,1,1.3-.25,3.55,3.55,0,0,1,1.31.25,3.22,3.22,0,0,1,1.09.71,3.39,3.39,0,0,1,.75,1.14,4.49,4.49,0,0,1,0,3.12A3.47,3.47,0,0,1,98,73.84a3.16,3.16,0,0,1-1.09.71A3.55,3.55,0,0,1,95.55,74.8Zm0-.91a2.37,2.37,0,0,0,.82-0.15,1.79,1.79,0,0,0,.71-0.48,2.48,2.48,0,0,0,.5-0.86,3.79,3.79,0,0,0,.19-1.27,3.73,3.73,0,0,0-.19-1.27,2.48,2.48,0,0,0-.5-0.85,1.79,1.79,0,0,0-.71-0.48,2.37,2.37,0,0,0-.82-0.15,2.25,2.25,0,0,0-.8.15,1.82,1.82,0,0,0-.69.48,2.4,2.4,0,0,0-.49.85,3.86,3.86,0,0,0-.18,1.27,3.92,3.92,0,0,0,.18,1.27,2.4,2.4,0,0,0,.49.86,1.81,1.81,0,0,0,.69.48A2.25,2.25,0,0,0,95.55,73.89Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M104.21,74.39a2.69,2.69,0,0,1-1.55.41,2.15,2.15,0,0,1-1.58-.6,2.49,2.49,0,0,1-.6-1.84V67.61h1.2v4.75a2.14,2.14,0,0,0,.1.7,1.08,1.08,0,0,0,.28.45,1,1,0,0,0,.42.24,1.82,1.82,0,0,0,.52.07,2,2,0,0,0,1.27-.38,3,3,0,0,0,.82-1V67.61h1.19v7h-1.05l-0.11-1.26h0A2.5,2.5,0,0,1,104.21,74.39Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M110.11,74.8a7.07,7.07,0,0,1-.9,0,5.45,5.45,0,0,1-.67-0.13,3.23,3.23,0,0,1-.51-0.17,3.55,3.55,0,0,1-.4-0.21l0.39-1a1.88,1.88,0,0,0,.28.19,3.34,3.34,0,0,0,.45.21,3.64,3.64,0,0,0,.61.18,3.74,3.74,0,0,0,.75.07,1.87,1.87,0,0,0,1.15-.3,1,1,0,0,0,.39-0.83,0.87,0.87,0,0,0-.32-0.71,3.35,3.35,0,0,0-1.11-.5l-0.86-.29a3,3,0,0,1-.73-0.38,1.8,1.8,0,0,1-.52-0.58,1.76,1.76,0,0,1-.2-0.88A1.68,1.68,0,0,1,108.6,68a3.23,3.23,0,0,1,1.89-.48,5.65,5.65,0,0,1,1.25.12,4.16,4.16,0,0,1,.84.27l-0.28,1a6.73,6.73,0,0,0-.7-0.3,3.05,3.05,0,0,0-1.06-.16,2.36,2.36,0,0,0-1.09.21,0.79,0.79,0,0,0-.41.77,0.73,0.73,0,0,0,.13.43,1.11,1.11,0,0,0,.34.31,2.48,2.48,0,0,0,.5.23l0.6,0.18,0.81,0.27a2.74,2.74,0,0,1,.72.39,1.74,1.74,0,0,1,.71,1.5,1.91,1.91,0,0,1-.23,1,2,2,0,0,1-.61.66,2.6,2.6,0,0,1-.87.38A4.33,4.33,0,0,1,110.11,74.8Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M119.76,74.05a3,3,0,0,1-1,.53,4.15,4.15,0,0,1-1.37.21,3.35,3.35,0,0,1-2.58-1,3.82,3.82,0,0,1-.9-2.71,4.38,4.38,0,0,1,.25-1.53,3.42,3.42,0,0,1,.69-1.15,2.91,2.91,0,0,1,1-.72,3.34,3.34,0,0,1,1.28-.25,3.2,3.2,0,0,1,1.28.25,2.31,2.31,0,0,1,.94.73,2.75,2.75,0,0,1,.5,1.2,5,5,0,0,1,0,1.66h-4.76q0.08,2.51,2.37,2.51a3.11,3.11,0,0,0,1.08-.17,4.26,4.26,0,0,0,.88-0.44Zm-2.59-5.74a1.77,1.77,0,0,0-1.29.54,2.71,2.71,0,0,0-.68,1.62h3.57a2.18,2.18,0,0,0-.37-1.63A1.53,1.53,0,0,0,117.17,68.31Z" transform="translate(-53.86 -64.51)"/></svg>

After

Width:  |  Height:  |  Size: 4.3 KiB

View File

@ -0,0 +1 @@
<svg xmlns="http://www.w3.org/2000/svg" viewBox="0 0 66.12 10.29"><defs><style>.a{fill:#fff;}</style></defs><title>ClickHouse white</title><path class="a" d="M58.54,74.8a5,5,0,0,1-1.88-.34,4.26,4.26,0,0,1-1.48-1,4.46,4.46,0,0,1-1-1.55,6.32,6.32,0,0,1,0-4.21,4.39,4.39,0,0,1,2.58-2.57,5.36,5.36,0,0,1,1.93-.34,5.84,5.84,0,0,1,1.56.18,6.38,6.38,0,0,1,1.2.46l-0.36,1.13A5.17,5.17,0,0,0,60,66a4.23,4.23,0,0,0-1.43-.22,3.07,3.07,0,0,0-2.35,1.06,3.81,3.81,0,0,0-.72,1.27,5.17,5.17,0,0,0-.27,1.73,5.35,5.35,0,0,0,.25,1.69,3.61,3.61,0,0,0,.69,1.25,2.88,2.88,0,0,0,1.06.77,3.49,3.49,0,0,0,1.38.27,4,4,0,0,0,1.27-.2A5.71,5.71,0,0,0,61,73.09l0.35,0.85a3.86,3.86,0,0,1-1.12.59A5,5,0,0,1,58.54,74.8Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M64.07,64.51V74.66H62.88V64.51h1.19Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M66.64,66.55a0.71,0.71,0,0,1-.76-0.74,0.78,0.78,0,0,1,.2-0.55,0.79,0.79,0,0,1,1.1,0,0.77,0.77,0,0,1,.21.55,0.72,0.72,0,0,1-.21.53A0.74,0.74,0,0,1,66.64,66.55Zm0.6,1.06v7H66.06v-7h1.19Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M72.29,74.8a3.78,3.78,0,0,1-1.37-.25,3.08,3.08,0,0,1-1.11-.72,3.45,3.45,0,0,1-.74-1.15,4.48,4.48,0,0,1,0-3.08,3.35,3.35,0,0,1,.76-1.15A3.19,3.19,0,0,1,71,67.72a4,4,0,0,1,1.41-.25,4.68,4.68,0,0,1,1.13.13,2.68,2.68,0,0,1,.84.35L74,69a4,4,0,0,0-.79-0.47,2.32,2.32,0,0,0-.92-0.16,2.28,2.28,0,0,0-.88.17,2,2,0,0,0-.73.52,2.52,2.52,0,0,0-.49.85A3.5,3.5,0,0,0,70,71.11a2.8,2.8,0,0,0,.66,2,2.4,2.4,0,0,0,1.79.67,3,3,0,0,0,1.62-.52l0.27,0.88a2.89,2.89,0,0,1-.89.45A3.88,3.88,0,0,1,72.29,74.8Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M77,64.51V74.66H75.79V64.51H77ZM80.1,74.66L77.16,70.9l2.9-3.28H81.3l-2.87,3.14,3.07,3.9H80.1Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M89.17,70.34h-5v4.31H82.88V64.93h1.29v4.44h5V64.93h1.29v9.73H89.17V70.34Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M95.55,74.8a3.49,3.49,0,0,1-1.3-.25,3.11,3.11,0,0,1-1.08-.71,3.44,3.44,0,0,1-.73-1.15,4.6,4.6,0,0,1,0-3.12,3.36,3.36,0,0,1,.73-1.14,3.16,3.16,0,0,1,1.08-.71,3.49,3.49,0,0,1,1.3-.25,3.55,3.55,0,0,1,1.31.25,3.22,3.22,0,0,1,1.09.71,3.39,3.39,0,0,1,.75,1.14,4.49,4.49,0,0,1,0,3.12A3.47,3.47,0,0,1,98,73.84a3.16,3.16,0,0,1-1.09.71A3.55,3.55,0,0,1,95.55,74.8Zm0-.91a2.37,2.37,0,0,0,.82-0.15,1.79,1.79,0,0,0,.71-0.48,2.48,2.48,0,0,0,.5-0.86,3.79,3.79,0,0,0,.19-1.27,3.73,3.73,0,0,0-.19-1.27,2.48,2.48,0,0,0-.5-0.85,1.79,1.79,0,0,0-.71-0.48,2.37,2.37,0,0,0-.82-0.15,2.25,2.25,0,0,0-.8.15,1.82,1.82,0,0,0-.69.48,2.4,2.4,0,0,0-.49.85,3.86,3.86,0,0,0-.18,1.27,3.92,3.92,0,0,0,.18,1.27,2.4,2.4,0,0,0,.49.86,1.81,1.81,0,0,0,.69.48A2.25,2.25,0,0,0,95.55,73.89Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M104.21,74.39a2.69,2.69,0,0,1-1.55.41,2.15,2.15,0,0,1-1.58-.6,2.49,2.49,0,0,1-.6-1.84V67.61h1.2v4.75a2.14,2.14,0,0,0,.1.7,1.08,1.08,0,0,0,.28.45,1,1,0,0,0,.42.24,1.82,1.82,0,0,0,.52.07,2,2,0,0,0,1.27-.38,3,3,0,0,0,.82-1V67.61h1.19v7h-1.05l-0.11-1.26h0A2.5,2.5,0,0,1,104.21,74.39Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M110.11,74.8a7.07,7.07,0,0,1-.9,0,5.45,5.45,0,0,1-.67-0.13,3.23,3.23,0,0,1-.51-0.17,3.55,3.55,0,0,1-.4-0.21l0.39-1a1.88,1.88,0,0,0,.28.19,3.34,3.34,0,0,0,.45.21,3.64,3.64,0,0,0,.61.18,3.74,3.74,0,0,0,.75.07,1.87,1.87,0,0,0,1.15-.3,1,1,0,0,0,.39-0.83,0.87,0.87,0,0,0-.32-0.71,3.35,3.35,0,0,0-1.11-.5l-0.86-.29a3,3,0,0,1-.73-0.38,1.8,1.8,0,0,1-.52-0.58,1.76,1.76,0,0,1-.2-0.88A1.68,1.68,0,0,1,108.6,68a3.23,3.23,0,0,1,1.89-.48,5.65,5.65,0,0,1,1.25.12,4.16,4.16,0,0,1,.84.27l-0.28,1a6.73,6.73,0,0,0-.7-0.3,3.05,3.05,0,0,0-1.06-.16,2.36,2.36,0,0,0-1.09.21,0.79,0.79,0,0,0-.41.77,0.73,0.73,0,0,0,.13.43,1.11,1.11,0,0,0,.34.31,2.48,2.48,0,0,0,.5.23l0.6,0.18,0.81,0.27a2.74,2.74,0,0,1,.72.39,1.74,1.74,0,0,1,.71,1.5,1.91,1.91,0,0,1-.23,1,2,2,0,0,1-.61.66,2.6,2.6,0,0,1-.87.38A4.33,4.33,0,0,1,110.11,74.8Z" transform="translate(-53.86 -64.51)"/><path class="a" d="M119.76,74.05a3,3,0,0,1-1,.53,4.15,4.15,0,0,1-1.37.21,3.35,3.35,0,0,1-2.58-1,3.82,3.82,0,0,1-.9-2.71,4.38,4.38,0,0,1,.25-1.53,3.42,3.42,0,0,1,.69-1.15,2.91,2.91,0,0,1,1-.72,3.34,3.34,0,0,1,1.28-.25,3.2,3.2,0,0,1,1.28.25,2.31,2.31,0,0,1,.94.73,2.75,2.75,0,0,1,.5,1.2,5,5,0,0,1,0,1.66h-4.76q0.08,2.51,2.37,2.51a3.11,3.11,0,0,0,1.08-.17,4.26,4.26,0,0,0,.88-0.44Zm-2.59-5.74a1.77,1.77,0,0,0-1.29.54,2.71,2.71,0,0,0-.68,1.62h3.57a2.18,2.18,0,0,0-.37-1.63A1.53,1.53,0,0,0,117.17,68.31Z" transform="translate(-53.86 -64.51)"/></svg>

After

Width:  |  Height:  |  Size: 4.3 KiB

View File

@ -1,71 +1,6 @@
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot);
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/CYblzLEXzCqQIvrYs7QKQe2omRk.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/pUcnOdRwl83MvPPzrNomhyletnA.woff) format('woff'),
url(https://yastatic.net/adv-www/_/vNFEmXOcGYKJ4AAidUprHWoXrLU.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/0w7OcWZM_QLP8x-LQUXFOgXO6dE.svg#YandexSansTextWeb-Bold) format('svg');
font-weight: 700;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot);
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/z3MYElcut0R2MF_Iw1RDNrstgYs.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/1jvKJ_-hCXl3s7gmFl-y_-UHTaI.woff) format('woff'),
url(https://yastatic.net/adv-www/_/9nzjfpCR2QHvK1EzHpDEIoVFGuY.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/gwyBTpxSwkFCF1looxqs6JokKls.svg#YandexSansTextWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot);
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/lGQcYklLVV0hyvz1HFmFsUTj8_0.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/f0AAJ9GJ4iiwEmhG-7PWMHk6vUY.woff) format('woff'),
url(https://yastatic.net/adv-www/_/4UDe4nlVvgEJ-VmLWNVq3SxCsA.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/EKLr1STNokPqxLAQa_RyN82pL98.svg#YandexSansTextWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot);
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/sUYVCPUAQE7ExrvMS7FoISoO83s.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/v2Sve_obH3rKm6rKrtSQpf-eB7U.woff) format('woff'),
url(https://yastatic.net/adv-www/_/PzD8hWLMunow5i3RfJ6WQJAL7aI.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/lF_KG5g4tpQNlYIgA0e77fBSZ5s.svg#YandexSansDisplayWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot);
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/LGiRvlfqQHlWR9YKLhsw5e7KGNA.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/40vXwNl4eYYMgteIVgLP49dwmfc.woff) format('woff'),
url(https://yastatic.net/adv-www/_/X6zG5x_wO8-AtwJ-vDLJcKC5228.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/ZKhaR0m08c8CRRL77GtFKoHcLYA.svg#YandexSansDisplayWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
body {
background: #fff;
font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif;
font: 300 14pt/200% -apple-system, BlinkMacSystemFont, Segoe UI, Helvetica, Arial, sans-serif, Apple Color Emoji,Segoe UI Emoji;
margin: 0;
padding: 0;
}
@ -93,8 +28,12 @@ body {
font-size: 110%;
}
#logo-text {
width: 200px;
height: 44px;
}
#main-title {
font: 400 42pt/90% 'Yandex Sans Display Web', Arial, sans-serif;
margin: 0;
}
@ -134,22 +73,21 @@ body {
h2 {
margin: 47px 0 23px;
font: 400 200%/133% 'Yandex Sans Display Web', Arial, sans-serif;
font-size: 200%;
}
a:link, a:visited {
color: #08f;
color: #f14600;
text-decoration: none;
}
a:hover, a:active {
color: #f00;
text-decoration: underline;
}
#top-menu {
margin: 12px 0 0 0;
font: 400 18pt 'Yandex Sans Display Web', Arial, sans-serif;
margin: 8px 0 0 0;
font-size: 18pt;
float: right;
}
@ -171,11 +109,10 @@ a:hover, a:active {
.index_item:active,
.index_item:visited {
color: #ededed;
font: 300 100% 'Yandex Sans Display Web', Arial, sans-serif;
}
#short-description {
font: 300 125%/150% 'Yandex Sans Display Web', Arial, sans-serif;
font-size: 125%;
margin: 0 1em 1.75em 0;
text-align: left;
}
@ -215,7 +152,7 @@ a:hover, a:active {
#announcement {
margin: 0 0 60px 0;
padding: 20px 0;
font: 400 125%/133% 'Yandex Sans Display Web',Arial,sans-serif;
font-size: 125%;
}
.announcement-link {
color: #000!important;
@ -268,7 +205,7 @@ a:hover, a:active {
}
pre {
font: 13px/18px monospace, "Courier New";
font: 13px/18px SFMono-Regular, Consolas, Liberation Mono, Menlo, monospace;
display: block;
border-left: 5px solid #ffdb4d;
padding: 5px 10px;
@ -313,9 +250,9 @@ img {
}
.distributive_not_selected {
color: #08f;
color: #f14600;
cursor: pointer;
border-bottom: 1px solid #08f;
border-bottom: 1px solid #f14600;
}
#tutorial_logo {
@ -326,7 +263,7 @@ img {
}
#tutorial_title {
font: normal 100px 'Yandex Sans Display Web', Arial, sans-serif;
font-size: 100px;
margin-top: 25px;
margin-bottom: 0;
text-align: center;
@ -349,8 +286,8 @@ img {
}
.spoiler_title {
color: #08f;
border-bottom: 1px dotted #08f;
color: #f14600;
border-bottom: 1px dotted #f14600;
}
.spoiler_title:hover {

View File

@ -47,7 +47,7 @@
<path class="orange" d="M8,3.25 h1 v1.5 h-1 z"></path>
</svg>
ClickHouse
<img id="logo-text" src="images/clickhouse-black.svg" alt="ClickHouse" />
</h1>
</a>
</div>
@ -450,7 +450,7 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
<a href="https://hub.docker.com/r/yandex/clickhouse-server/" rel="external nofollow"
target="_blank">
official Docker images of ClickHouse</a>, this is not the only <a href="https://clickhouse.tech/docs/en/getting_started/">option</a> though.
Alternatively, you can easily get a running ClickHouse instance or cluster at
Alternatively, you can easily get a running ClickHouse instance or cluster at
<a href="https://cloud.yandex.com/services/managed-clickhouse?utm_source=referrals&utm_medium=clickhouseofficialsite&utm_campaign=link1" rel="external nofollow" target="_blank">
Yandex Managed Service for ClickHouse</a>.
</p>
@ -529,15 +529,15 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
line = line.split('](');
var tail = line[1].split(') ');
result.push(
'<a class="announcement-link" rel="external nofollow" target="_blank" href="' +
tail[0] + '">' + line[0].replace('* [', '').replace('ClickHouse Meetup in ', '') +
'<a class="announcement-link" rel="external nofollow" target="_blank" href="' +
tail[0] + '">' + line[0].replace('* [', '').replace('ClickHouse Meetup in ', '') +
'</a> ' + tail[1].slice(0, -1)
);
}
});
if (result.length) {
if (result.length == 1) {
result = 'Upcoming Meetup: ' + result[0];
result = 'Upcoming Meetup: ' + result[0];
} else {
result = 'Upcoming Meetups: ' + result.join(', ');
var offset = result.lastIndexOf(', ');